From 164489d6f176bdecfa9dabec2dfce5504d1ee8af Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 3 Feb 2014 18:18:04 -0800 Subject: [PATCH 01/68] Relax assumptions on compressors and serializers when batching This commit introduces an intermediate layer of an input stream on the batch level. This guards against interference from higher level streams (i.e. compression and deserialization streams), especially pre-fetching, without specifically targeting particular libraries (Kryo) and forcing shuffle spill compression to use LZF. --- .../spark/storage/BlockObjectWriter.scala | 8 +- .../collection/ExternalAppendOnlyMap.scala | 166 +++++++++--------- 2 files changed, 90 insertions(+), 84 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 530712b5df4a8..696b930a26b9e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -66,6 +66,11 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { * Cumulative time spent performing blocking writes, in ns. */ def timeWriting(): Long + + /** + * Number of bytes written so far + */ + def bytesWritten: Long } /** BlockObjectWriter which writes directly to a file on disk. Appends to the given file. */ @@ -183,7 +188,8 @@ private[spark] class DiskBlockObjectWriter( // Only valid if called after close() override def timeWriting() = _timeWriting - def bytesWritten: Long = { + // Only valid if called after commit() + override def bytesWritten: Long = { lastValidPosition - initialPosition } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 3d9b09ec33e2a..a3dcdd7d968a1 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -26,9 +26,8 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.io.LZFCompressionCodec -import org.apache.spark.serializer.{KryoDeserializationStream, Serializer} -import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockObjectWriter} +import org.apache.spark.serializer.Serializer +import org.apache.spark.storage.{BlockId, BlockManager} /** * An append-only map that spills sorted content to disk when there is insufficient space for it @@ -84,12 +83,14 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Number of in-memory pairs inserted before tracking the map's shuffle memory usage private val trackMemoryThreshold = 1000 - // Size of object batches when reading/writing from serializers. Objects are written in - // batches, with each batch using its own serialization stream. This cuts down on the size - // of reference-tracking maps constructed when deserializing a stream. - // - // NOTE: Setting this too low can cause excess copying when serializing, since some serializers - // grow internal data structures by growing + copying every time the number of objects doubles. + /* Size of object batches when reading/writing from serializers. + * + * Objects are written in batches, with each batch using its own serialization stream. This + * cuts down on the size of reference-tracking maps constructed when deserializing a stream. + * + * NOTE: Setting this too low can cause excess copying when serializing, since some serializers + * grow internal data structures by growing + copying every time the number of objects doubles. + */ private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) // How many times we have spilled so far @@ -100,7 +101,6 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 - private val syncWrites = sparkConf.getBoolean("spark.shuffle.sync", false) private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() @@ -153,37 +153,21 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( logWarning("Spilling in-memory map of %d MB to disk (%d time%s so far)" .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() + var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) + var objectsWritten = 0 - /* IMPORTANT NOTE: To avoid having to keep large object graphs in memory, this approach - * closes and re-opens serialization and compression streams within each file. This makes some - * assumptions about the way that serialization and compression streams work, specifically: - * - * 1) The serializer input streams do not pre-fetch data from the underlying stream. - * - * 2) Several compression streams can be opened, written to, and flushed on the write path - * while only one compression input stream is created on the read path - * - * In practice (1) is only true for Java, so we add a special fix below to make it work for - * Kryo. (2) is only true for LZF and not Snappy, so we coerce this to use LZF. - * - * To avoid making these assumptions we should create an intermediate stream that batches - * objects and sends an EOF to the higher layer streams to make sure they never prefetch data. - * This is a bit tricky because, within each segment, you'd need to track the total number - * of bytes written and then re-wind and write it at the beginning of the segment. This will - * most likely require using the file channel API. - */ + // List of batch sizes (bytes) in the order they are written to disk + val batchSizes = new ArrayBuffer[Long] - val shouldCompress = blockManager.shouldCompress(blockId) - val compressionCodec = new LZFCompressionCodec(sparkConf) - def wrapForCompression(outputStream: OutputStream) = { - if (shouldCompress) compressionCodec.compressedOutputStream(outputStream) else outputStream + // Flush the disk writer's contents to disk, and update relevant variables + def flush() = { + writer.commit() + val bytesWritten = writer.bytesWritten + batchSizes.append(bytesWritten) + _diskBytesSpilled += bytesWritten + objectsWritten = 0 } - def getNewWriter = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, - wrapForCompression, syncWrites) - - var writer = getNewWriter - var objectsWritten = 0 try { val it = currentMap.destructiveSortedIterator(comparator) while (it.hasNext) { @@ -192,22 +176,21 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( objectsWritten += 1 if (objectsWritten == serializerBatchSize) { - writer.commit() + flush() writer.close() - _diskBytesSpilled += writer.bytesWritten - writer = getNewWriter - objectsWritten = 0 + writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) } } - - if (objectsWritten > 0) writer.commit() + if (objectsWritten > 0) { + flush() + } } finally { // Partial failures cannot be tolerated; do not revert partial writes writer.close() - _diskBytesSpilled += writer.bytesWritten } + currentMap = new SizeTrackingAppendOnlyMap[K, C] - spilledMaps.append(new DiskMapIterator(file, blockId)) + spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) // Reset the amount of shuffle memory used by this map in the global pool val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap @@ -252,8 +235,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } /** - * Fetch from the given iterator until a key of different hash is retrieved. In the - * event of key hash collisions, this ensures no pairs are hidden from being merged. + * Fetch from the given iterator until a key of different hash is retrieved. + * + * In the event of key hash collisions, this ensures no pairs are hidden from being merged. * Assume the given iterator is in sorted order. */ def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { @@ -293,7 +277,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( override def hasNext: Boolean = mergeHeap.exists(!_.pairs.isEmpty) /** - * Select a key with the minimum hash, then combine all values with the same key from all input streams. + * Select a key with the minimum hash, then combine all values with the same key from all + * input streams */ override def next(): (K, C) = { // Select a key from the StreamBuffer that holds the lowest key hash @@ -355,51 +340,66 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * An iterator that returns (K, C) pairs in sorted order from an on-disk map */ - private class DiskMapIterator(file: File, blockId: BlockId) extends Iterator[(K, C)] { + private class DiskMapIterator(file: File, + blockId: BlockId, + batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) - val shouldCompress = blockManager.shouldCompress(blockId) - val compressionCodec = new LZFCompressionCodec(sparkConf) - val compressedStream = - if (shouldCompress) { - compressionCodec.compressedInputStream(bufferedStream) - } else { - bufferedStream - } - var deserializeStream = ser.deserializeStream(compressedStream) - var objectsRead = 0 + // An intermediate stream that holds all the bytes from exactly one batch + // This guards against pre-fetching and other arbitrary behavior of higher level streams + var batchStream = nextBatchStream(bufferedStream) + var compressedStream = blockManager.wrapForCompression(blockId, batchStream) + var deserializeStream = ser.deserializeStream(compressedStream) var nextItem: (K, C) = null var eof = false + /** + * Construct a stream that contains all the bytes from the next batch + */ + def nextBatchStream(stream: InputStream): ByteArrayInputStream = { + var batchBytes = Array[Byte]() + if (batchSizes.length > 0) { + val batchSize = batchSizes.remove(0) + + // Read batchSize number of bytes into batchBytes + while (batchBytes.length < batchSize) { + val numBytesToRead = Math.min(8192, batchSize - batchBytes.length).toInt + val bytesRead = new Array[Byte](numBytesToRead) + stream.read(bytesRead, 0, numBytesToRead) + batchBytes ++= bytesRead + } + } else { + // No more batches left + eof = true + } + new ByteArrayInputStream(batchBytes) + } + + /** + * Return the next (K, C) pair from the deserialization stream. + * + * If the underlying batch stream is drained, construct a new stream for the next batch + * (if there is one) and stream from it. If there are no more batches left, return null. + */ def readNextItem(): (K, C) = { - if (!eof) { - try { - if (objectsRead == serializerBatchSize) { - val newInputStream = deserializeStream match { - case stream: KryoDeserializationStream => - // Kryo's serializer stores an internal buffer that pre-fetches from the underlying - // stream. We need to capture this buffer and feed it to the new serialization - // stream so that the bytes are not lost. - val kryoInput = stream.input - val remainingBytes = kryoInput.limit() - kryoInput.position() - val extraBuf = kryoInput.readBytes(remainingBytes) - new SequenceInputStream(new ByteArrayInputStream(extraBuf), compressedStream) - case _ => compressedStream - } - deserializeStream = ser.deserializeStream(newInputStream) - objectsRead = 0 - } - objectsRead += 1 - return deserializeStream.readObject().asInstanceOf[(K, C)] - } catch { - case e: EOFException => - eof = true + try { + deserializeStream.readObject().asInstanceOf[(K, C)] + } catch { + // End of current batch + case e: EOFException => + batchStream = nextBatchStream(bufferedStream) + if (!eof) { + compressedStream = blockManager.wrapForCompression(blockId, batchStream) + deserializeStream = ser.deserializeStream(compressedStream) + readNextItem() + } else { + // No more batches left cleanup() - } + null + } } - null } override def hasNext: Boolean = { From a531d2e347acdcecf2d0ab72cd4f965ab5e145d8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 3 Feb 2014 18:18:04 -0800 Subject: [PATCH 02/68] Relax assumptions on compressors and serializers when batching This commit introduces an intermediate layer of an input stream on the batch level. This guards against interference from higher level streams (i.e. compression and deserialization streams), especially pre-fetching, without specifically targeting particular libraries (Kryo) and forcing shuffle spill compression to use LZF. --- .../spark/storage/BlockObjectWriter.scala | 8 +- .../collection/ExternalAppendOnlyMap.scala | 166 +++++++++--------- 2 files changed, 90 insertions(+), 84 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 530712b5df4a8..696b930a26b9e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -66,6 +66,11 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { * Cumulative time spent performing blocking writes, in ns. */ def timeWriting(): Long + + /** + * Number of bytes written so far + */ + def bytesWritten: Long } /** BlockObjectWriter which writes directly to a file on disk. Appends to the given file. */ @@ -183,7 +188,8 @@ private[spark] class DiskBlockObjectWriter( // Only valid if called after close() override def timeWriting() = _timeWriting - def bytesWritten: Long = { + // Only valid if called after commit() + override def bytesWritten: Long = { lastValidPosition - initialPosition } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 3d9b09ec33e2a..a3dcdd7d968a1 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -26,9 +26,8 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.io.LZFCompressionCodec -import org.apache.spark.serializer.{KryoDeserializationStream, Serializer} -import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockObjectWriter} +import org.apache.spark.serializer.Serializer +import org.apache.spark.storage.{BlockId, BlockManager} /** * An append-only map that spills sorted content to disk when there is insufficient space for it @@ -84,12 +83,14 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Number of in-memory pairs inserted before tracking the map's shuffle memory usage private val trackMemoryThreshold = 1000 - // Size of object batches when reading/writing from serializers. Objects are written in - // batches, with each batch using its own serialization stream. This cuts down on the size - // of reference-tracking maps constructed when deserializing a stream. - // - // NOTE: Setting this too low can cause excess copying when serializing, since some serializers - // grow internal data structures by growing + copying every time the number of objects doubles. + /* Size of object batches when reading/writing from serializers. + * + * Objects are written in batches, with each batch using its own serialization stream. This + * cuts down on the size of reference-tracking maps constructed when deserializing a stream. + * + * NOTE: Setting this too low can cause excess copying when serializing, since some serializers + * grow internal data structures by growing + copying every time the number of objects doubles. + */ private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) // How many times we have spilled so far @@ -100,7 +101,6 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 - private val syncWrites = sparkConf.getBoolean("spark.shuffle.sync", false) private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() @@ -153,37 +153,21 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( logWarning("Spilling in-memory map of %d MB to disk (%d time%s so far)" .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() + var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) + var objectsWritten = 0 - /* IMPORTANT NOTE: To avoid having to keep large object graphs in memory, this approach - * closes and re-opens serialization and compression streams within each file. This makes some - * assumptions about the way that serialization and compression streams work, specifically: - * - * 1) The serializer input streams do not pre-fetch data from the underlying stream. - * - * 2) Several compression streams can be opened, written to, and flushed on the write path - * while only one compression input stream is created on the read path - * - * In practice (1) is only true for Java, so we add a special fix below to make it work for - * Kryo. (2) is only true for LZF and not Snappy, so we coerce this to use LZF. - * - * To avoid making these assumptions we should create an intermediate stream that batches - * objects and sends an EOF to the higher layer streams to make sure they never prefetch data. - * This is a bit tricky because, within each segment, you'd need to track the total number - * of bytes written and then re-wind and write it at the beginning of the segment. This will - * most likely require using the file channel API. - */ + // List of batch sizes (bytes) in the order they are written to disk + val batchSizes = new ArrayBuffer[Long] - val shouldCompress = blockManager.shouldCompress(blockId) - val compressionCodec = new LZFCompressionCodec(sparkConf) - def wrapForCompression(outputStream: OutputStream) = { - if (shouldCompress) compressionCodec.compressedOutputStream(outputStream) else outputStream + // Flush the disk writer's contents to disk, and update relevant variables + def flush() = { + writer.commit() + val bytesWritten = writer.bytesWritten + batchSizes.append(bytesWritten) + _diskBytesSpilled += bytesWritten + objectsWritten = 0 } - def getNewWriter = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, - wrapForCompression, syncWrites) - - var writer = getNewWriter - var objectsWritten = 0 try { val it = currentMap.destructiveSortedIterator(comparator) while (it.hasNext) { @@ -192,22 +176,21 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( objectsWritten += 1 if (objectsWritten == serializerBatchSize) { - writer.commit() + flush() writer.close() - _diskBytesSpilled += writer.bytesWritten - writer = getNewWriter - objectsWritten = 0 + writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) } } - - if (objectsWritten > 0) writer.commit() + if (objectsWritten > 0) { + flush() + } } finally { // Partial failures cannot be tolerated; do not revert partial writes writer.close() - _diskBytesSpilled += writer.bytesWritten } + currentMap = new SizeTrackingAppendOnlyMap[K, C] - spilledMaps.append(new DiskMapIterator(file, blockId)) + spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) // Reset the amount of shuffle memory used by this map in the global pool val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap @@ -252,8 +235,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } /** - * Fetch from the given iterator until a key of different hash is retrieved. In the - * event of key hash collisions, this ensures no pairs are hidden from being merged. + * Fetch from the given iterator until a key of different hash is retrieved. + * + * In the event of key hash collisions, this ensures no pairs are hidden from being merged. * Assume the given iterator is in sorted order. */ def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { @@ -293,7 +277,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( override def hasNext: Boolean = mergeHeap.exists(!_.pairs.isEmpty) /** - * Select a key with the minimum hash, then combine all values with the same key from all input streams. + * Select a key with the minimum hash, then combine all values with the same key from all + * input streams */ override def next(): (K, C) = { // Select a key from the StreamBuffer that holds the lowest key hash @@ -355,51 +340,66 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * An iterator that returns (K, C) pairs in sorted order from an on-disk map */ - private class DiskMapIterator(file: File, blockId: BlockId) extends Iterator[(K, C)] { + private class DiskMapIterator(file: File, + blockId: BlockId, + batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) - val shouldCompress = blockManager.shouldCompress(blockId) - val compressionCodec = new LZFCompressionCodec(sparkConf) - val compressedStream = - if (shouldCompress) { - compressionCodec.compressedInputStream(bufferedStream) - } else { - bufferedStream - } - var deserializeStream = ser.deserializeStream(compressedStream) - var objectsRead = 0 + // An intermediate stream that holds all the bytes from exactly one batch + // This guards against pre-fetching and other arbitrary behavior of higher level streams + var batchStream = nextBatchStream(bufferedStream) + var compressedStream = blockManager.wrapForCompression(blockId, batchStream) + var deserializeStream = ser.deserializeStream(compressedStream) var nextItem: (K, C) = null var eof = false + /** + * Construct a stream that contains all the bytes from the next batch + */ + def nextBatchStream(stream: InputStream): ByteArrayInputStream = { + var batchBytes = Array[Byte]() + if (batchSizes.length > 0) { + val batchSize = batchSizes.remove(0) + + // Read batchSize number of bytes into batchBytes + while (batchBytes.length < batchSize) { + val numBytesToRead = Math.min(8192, batchSize - batchBytes.length).toInt + val bytesRead = new Array[Byte](numBytesToRead) + stream.read(bytesRead, 0, numBytesToRead) + batchBytes ++= bytesRead + } + } else { + // No more batches left + eof = true + } + new ByteArrayInputStream(batchBytes) + } + + /** + * Return the next (K, C) pair from the deserialization stream. + * + * If the underlying batch stream is drained, construct a new stream for the next batch + * (if there is one) and stream from it. If there are no more batches left, return null. + */ def readNextItem(): (K, C) = { - if (!eof) { - try { - if (objectsRead == serializerBatchSize) { - val newInputStream = deserializeStream match { - case stream: KryoDeserializationStream => - // Kryo's serializer stores an internal buffer that pre-fetches from the underlying - // stream. We need to capture this buffer and feed it to the new serialization - // stream so that the bytes are not lost. - val kryoInput = stream.input - val remainingBytes = kryoInput.limit() - kryoInput.position() - val extraBuf = kryoInput.readBytes(remainingBytes) - new SequenceInputStream(new ByteArrayInputStream(extraBuf), compressedStream) - case _ => compressedStream - } - deserializeStream = ser.deserializeStream(newInputStream) - objectsRead = 0 - } - objectsRead += 1 - return deserializeStream.readObject().asInstanceOf[(K, C)] - } catch { - case e: EOFException => - eof = true + try { + deserializeStream.readObject().asInstanceOf[(K, C)] + } catch { + // End of current batch + case e: EOFException => + batchStream = nextBatchStream(bufferedStream) + if (!eof) { + compressedStream = blockManager.wrapForCompression(blockId, batchStream) + deserializeStream = ser.deserializeStream(compressedStream) + readNextItem() + } else { + // No more batches left cleanup() - } + null + } } - null } override def hasNext: Boolean = { From 287ef44e593ad72f7434b759be3170d9ee2723d2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Feb 2014 13:38:32 -0800 Subject: [PATCH 03/68] Avoid reading the entire batch into memory; also simplify streaming logic Additionally, address formatting comments. --- .../collection/ExternalAppendOnlyMap.scala | 63 ++++++++----------- 1 file changed, 26 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index a3dcdd7d968a1..bd35f9a16bb4d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedInputStream +import com.google.common.io.ByteStreams import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer @@ -83,12 +84,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Number of in-memory pairs inserted before tracking the map's shuffle memory usage private val trackMemoryThreshold = 1000 - /* Size of object batches when reading/writing from serializers. + /** + * Size of object batches when reading/writing from serializers. * * Objects are written in batches, with each batch using its own serialization stream. This * cuts down on the size of reference-tracking maps constructed when deserializing a stream. * - * NOTE: Setting this too low can cause excess copying when serializing, since some serializers + * NOTE: Setting this too low can cause excessive copying when serializing, since some serializers * grow internal data structures by growing + copying every time the number of objects doubles. */ private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) @@ -340,65 +342,52 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * An iterator that returns (K, C) pairs in sorted order from an on-disk map */ - private class DiskMapIterator(file: File, - blockId: BlockId, - batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { + private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) + extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) - // An intermediate stream that holds all the bytes from exactly one batch + // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams - var batchStream = nextBatchStream(bufferedStream) - + var batchStream = nextBatchStream() var compressedStream = blockManager.wrapForCompression(blockId, batchStream) var deserializeStream = ser.deserializeStream(compressedStream) var nextItem: (K, C) = null - var eof = false + var objectsRead = 0 /** - * Construct a stream that contains all the bytes from the next batch + * Construct a stream that reads only from the next batch */ - def nextBatchStream(stream: InputStream): ByteArrayInputStream = { - var batchBytes = Array[Byte]() + def nextBatchStream(): InputStream = { if (batchSizes.length > 0) { - val batchSize = batchSizes.remove(0) - - // Read batchSize number of bytes into batchBytes - while (batchBytes.length < batchSize) { - val numBytesToRead = Math.min(8192, batchSize - batchBytes.length).toInt - val bytesRead = new Array[Byte](numBytesToRead) - stream.read(bytesRead, 0, numBytesToRead) - batchBytes ++= bytesRead - } + ByteStreams.limit(bufferedStream, batchSizes.remove(0)) } else { // No more batches left - eof = true + bufferedStream } - new ByteArrayInputStream(batchBytes) } /** * Return the next (K, C) pair from the deserialization stream. * - * If the underlying batch stream is drained, construct a new stream for the next batch - * (if there is one) and stream from it. If there are no more batches left, return null. + * If the current batch is drained, construct a stream for the next batch and read from it. + * If no more pairs are left, return null. */ def readNextItem(): (K, C) = { try { - deserializeStream.readObject().asInstanceOf[(K, C)] + val item = deserializeStream.readObject().asInstanceOf[(K, C)] + objectsRead += 1 + if (objectsRead == serializerBatchSize) { + batchStream = nextBatchStream() + compressedStream = blockManager.wrapForCompression(blockId, batchStream) + deserializeStream = ser.deserializeStream(compressedStream) + objectsRead = 0 + } + item } catch { - // End of current batch case e: EOFException => - batchStream = nextBatchStream(bufferedStream) - if (!eof) { - compressedStream = blockManager.wrapForCompression(blockId, batchStream) - deserializeStream = ser.deserializeStream(compressedStream) - readNextItem() - } else { - // No more batches left - cleanup() - null - } + cleanup() + null } } From bd5a1d7350467ed3dc19c2de9b2c9f531f0e6aa3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Feb 2014 13:44:24 -0800 Subject: [PATCH 04/68] Typo: phyiscal -> physical --- .../main/scala/org/apache/spark/storage/DiskBlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index a8ef7fa8b63eb..f3e1c38744d78 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -50,7 +50,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD addShutdownHook() /** - * Returns the phyiscal file segment in which the given BlockId is located. + * Returns the physical file segment in which the given BlockId is located. * If the BlockId has been mapped to a specific FileSegment, that will be returned. * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly. */ From 13920c918efe22e66a1760b14beceb17a61fd8cc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Feb 2014 16:34:15 -0800 Subject: [PATCH 05/68] Update docs --- docs/configuration.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 1f9fa7056697e..8e4c48c81f8be 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -158,9 +158,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.spill.compress true - Whether to compress data spilled during shuffles. If enabled, spill compression - always uses the `org.apache.spark.io.LZFCompressionCodec` codec, - regardless of the value of `spark.io.compression.codec`. + Whether to compress data spilled during shuffles. From 090544a87a0767effd0c835a53952f72fc8d24f0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Feb 2014 10:58:23 -0800 Subject: [PATCH 06/68] Privatize methods --- .../util/collection/ExternalAppendOnlyMap.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index bd35f9a16bb4d..ccad5d3441954 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -242,7 +242,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * In the event of key hash collisions, this ensures no pairs are hidden from being merged. * Assume the given iterator is in sorted order. */ - def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { + private def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { val kcPairs = new ArrayBuffer[(K, C)] if (it.hasNext) { var kc = it.next() @@ -260,7 +260,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * If the given buffer contains a value for the given key, merge that value into * baseCombiner and remove the corresponding (K, C) pair from the buffer */ - def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { + private def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { var i = 0 while (i < buffer.pairs.size) { val (k, c) = buffer.pairs(i) @@ -320,7 +320,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ - case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) + private case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def minKeyHash: Int = { @@ -358,7 +358,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * Construct a stream that reads only from the next batch */ - def nextBatchStream(): InputStream = { + private def nextBatchStream(): InputStream = { if (batchSizes.length > 0) { ByteStreams.limit(bufferedStream, batchSizes.remove(0)) } else { @@ -373,7 +373,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * If the current batch is drained, construct a stream for the next batch and read from it. * If no more pairs are left, return null. */ - def readNextItem(): (K, C) = { + private def readNextItem(): (K, C) = { try { val item = deserializeStream.readObject().asInstanceOf[(K, C)] objectsRead += 1 @@ -408,7 +408,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } // TODO: Ensure this gets called even if the iterator isn't drained. - def cleanup() { + private def cleanup() { deserializeStream.close() file.delete() } From 3ddeb7ef89a0af2b685fb5d071aa0f71c975cc82 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Feb 2014 12:09:32 -0800 Subject: [PATCH 07/68] Also privatize fields --- .../collection/ExternalAppendOnlyMap.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ccad5d3441954..7eb300d46e6e2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -224,12 +224,12 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging - val mergeHeap = new mutable.PriorityQueue[StreamBuffer] + private val mergeHeap = new mutable.PriorityQueue[StreamBuffer] // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order - val sortedMap = currentMap.destructiveSortedIterator(comparator) - val inputStreams = Seq(sortedMap) ++ spilledMaps + private val sortedMap = currentMap.destructiveSortedIterator(comparator) + private val inputStreams = Seq(sortedMap) ++ spilledMaps inputStreams.foreach { it => val kcPairs = getMorePairs(it) @@ -344,16 +344,16 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( */ private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { - val fileStream = new FileInputStream(file) - val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) + private val fileStream = new FileInputStream(file) + private val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams - var batchStream = nextBatchStream() - var compressedStream = blockManager.wrapForCompression(blockId, batchStream) - var deserializeStream = ser.deserializeStream(compressedStream) - var nextItem: (K, C) = null - var objectsRead = 0 + private var batchStream = nextBatchStream() + private var compressedStream = blockManager.wrapForCompression(blockId, batchStream) + private var deserializeStream = ser.deserializeStream(compressedStream) + private var nextItem: (K, C) = null + private var objectsRead = 0 /** * Construct a stream that reads only from the next batch From 8e09306f6dd4ab421447d769572de58035d3d66a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 11 Feb 2014 17:48:16 -0800 Subject: [PATCH 08/68] Use JSON for ExecutorsUI --- .../org/apache/spark/scheduler/TaskInfo.scala | 2 - .../apache/spark/ui/exec/ExecutorsUI.scala | 118 +++++++++++------- .../spark/ui/jobs/JobProgressListener.scala | 1 - .../org/apache/spark/util/FileLogger.scala | 89 +++++++++++++ 4 files changed, 159 insertions(+), 51 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/FileLogger.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 91c27d7b8e9d7..6183b125def99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.util.Utils - /** * Information about a running task attempt inside a TaskSet. */ diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 4e41acf0230f9..3f5ce2f68eba5 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -19,24 +19,27 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.server.Handler import org.apache.spark.{ExceptionFailure, Logging, SparkContext} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} -import org.apache.spark.scheduler.TaskInfo +import org.apache.spark.scheduler.{SparkListenerJobEnd, SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, FileLogger} +import net.liftweb.json.DefaultFormats +import net.liftweb.json.JsonAST._ +import net.liftweb.json.JsonDSL._ private[spark] class ExecutorsUI(val sc: SparkContext) { private var _listener: Option[ExecutorsListener] = None + private implicit val format = DefaultFormats + def listener = _listener.get def start() { @@ -111,13 +114,13 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = status.memUsed().toString val maxMem = status.maxMem.toString val diskUsed = status.diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(execId, HashSet.empty[Long]).size - val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) - val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) - val totalTasks = activeTasks + failedTasks + completedTasks - val totalDuration = listener.executorToDuration.getOrElse(execId, 0) - val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) - val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) + val activeTasks = listener.getJson(execId, "Active Tasks").extract[Int] + val failedTasks = listener.getJson(execId, "Failed Tasks").extract[Int] + val completeTasks = listener.getJson(execId, "Complete Tasks").extract[Int] + val totalTasks = activeTasks + failedTasks + completeTasks + val totalDuration = listener.getJson(execId, "Task Time").extract[Long] + val totalShuffleRead = listener.getJson(execId, "Shuffle Read").extract[Long] + val totalShuffleWrite = listener.getJson(execId, "Shuffle Write").extract[Long] Seq( execId, @@ -128,7 +131,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { diskUsed, activeTasks.toString, failedTasks.toString, - completedTasks.toString, + completeTasks.toString, totalTasks.toString, totalDuration.toString, totalShuffleRead.toString, @@ -137,46 +140,65 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - override def onTaskStart(taskStart: SparkListenerTaskStart) { + val executorIdToJson = HashMap[String, JValue]() + val logger = new FileLogger("executors-ui") + + def newJson(execId: String): JValue = { + ("Executor ID" -> execId) ~ + ("Active Tasks" -> 0) ~ + ("Failed Tasks" -> 0) ~ + ("Complete Tasks" -> 0) ~ + ("Task Time" -> 0L) ~ + ("Shuffle Read" -> 0L) ~ + ("Shuffle Write" -> 0L) + } + + def getJson(execId: String, field: String): JValue = { + executorIdToJson.get(execId) match { + case Some(json) => (json \ field) + case None => JNothing + } + } + + def logJson(json: JValue) = logger.logLine(compactRender(json)) + + override def onTaskStart(taskStart: SparkListenerTaskStart) = { val eid = taskStart.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks += taskStart.taskInfo + var json = executorIdToJson.getOrElseUpdate(eid, newJson(eid)) + json = json.transform { + case JField("Active Tasks", JInt(s)) => JField("Active Tasks", JInt(s + 1)) + } + executorIdToJson(eid) = json + logJson(json) } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = { val eid = taskEnd.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration - executorToDuration.put(eid, newDuration) - - activeTasks -= taskEnd.taskInfo - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - (Some(e), e.metrics) - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } - - // update shuffle read/write - if (null != taskEnd.taskMetrics) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach(shuffleRead => - executorToShuffleRead.put(eid, executorToShuffleRead.getOrElse(eid, 0L) + - shuffleRead.remoteBytesRead)) - - taskEnd.taskMetrics.shuffleWriteMetrics.foreach(shuffleWrite => - executorToShuffleWrite.put(eid, executorToShuffleWrite.getOrElse(eid, 0L) + - shuffleWrite.shuffleBytesWritten)) + val exception = taskEnd.reason match { + case _: ExceptionFailure => true + case _ => false + } + val newDuration = taskEnd.taskInfo.duration + var newShuffleRead = 0 + var newShuffleWrite = 0 + if (taskEnd.taskMetrics != null) { + taskEnd.taskMetrics.shuffleReadMetrics.foreach(newShuffleRead += _.remoteBytesRead) + taskEnd.taskMetrics.shuffleWriteMetrics.foreach(newShuffleWrite += _.shuffleBytesWritten) } + var json = executorIdToJson.getOrElseUpdate(eid, newJson(eid)) + json = json.transform { + case JField("Active Tasks", JInt(s)) if s > 0 => JField("Active Tasks", JInt(s - 1)) + case JField("Failed Tasks", JInt(s)) if exception => JField("Failed Tasks", JInt(s + 1)) + case JField("Complete Tasks", JInt(s)) if !exception => + JField("Complete Tasks", JInt(s + 1)) + case JField("Task Time", JInt(s)) => JField("Task Time", JInt(s + newDuration)) + case JField("Shuffle Read", JInt(s)) => JField("Shuffle Read", JInt(s + newShuffleRead)) + case JField("Shuffle Write", JInt(s)) => JField("Shuffle Write", JInt(s + newShuffleWrite)) + } + executorIdToJson(eid) = json + logJson(json) } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 858a10ce750ff..07a08f5277d19 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} import org.apache.spark.{ExceptionFailure, SparkContext, Success} diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala new file mode 100644 index 0000000000000..2afc49f2e4330 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -0,0 +1,89 @@ +/* + * 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.util + +import java.io.{IOException, File, PrintWriter} +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.spark._ + +/** + * A generic class for logging information to file + */ + +class FileLogger(user: String, name: String, flushFrequency: Int = 1) extends Logging { + + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private var logCount = 0 + + private val logDir = + if (System.getenv("SPARK_LOG_DIR") != null) { + System.getenv("SPARK_LOG_DIR") + } else { + "/tmp/spark-%s".format(user) + } + + private val logFile = logDir + "/" + name + + private val writer: PrintWriter = { + createLogDir() + new PrintWriter(logFile) + } + + def this() = this(System.getProperty("user.name", ""), + String.valueOf(System.currentTimeMillis())) + + def this(_name: String) = this(System.getProperty("user.name", ""), _name) + + /** Create a logging directory with the given path */ + private def createLogDir() { + val dir = new File(logDir) + if (!dir.exists && !dir.mkdirs()) { + // Logger should throw a exception rather than continue to construct this object + throw new IOException("create log directory error:" + logDir) + } + val file = new File(logFile) + if (file.exists) { + logWarning("Overwriting existing log file at %s".format(logFile)) + } + } + + /** Log the message to the given writer if it exists, optionally including the time */ + def log(msg: String, withTime: Boolean = false) = { + var writeInfo = msg + if (withTime) { + val date = new Date(System.currentTimeMillis()) + writeInfo = DATE_FORMAT.format(date) + ": " + msg + } + writer.print(writeInfo) + logCount += 1 + if (logCount % flushFrequency == 0) { + writer.flush() + logCount = 0 + } + } + + /** + * Log the message as a new line to the given writer if it exists, optionally including the time + */ + def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) + + /** Close the writer, if it exists */ + def close() = writer.close() +} From dcbd312b1e4585445868dfb562f9c64ac2fc8cda Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 12 Feb 2014 15:58:39 -0800 Subject: [PATCH 09/68] Add JSON Serializability for all SparkListenerEvent's This also involves a clean-up in the way these events are structured. The existing way in which these events are defined maintains a lot of extraneous information. To avoid serializing the whole tree of RDD dependencies, for instance, this commit cherry-picks only the relevant fields. However, this means sacrificing JobLogger's functionality of tracing the entire RDD tree. Additionally, this commit also involves minor formatting and naming clean-ups within the scope of the above changes. --- .../apache/spark/executor/TaskMetrics.scala | 99 +++--- .../apache/spark/scheduler/DAGScheduler.scala | 25 +- .../apache/spark/scheduler/JobLogger.scala | 286 +++++------------- .../apache/spark/scheduler/JobResult.scala | 22 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../spark/scheduler/SparkListener.scala | 125 +++++--- .../spark/scheduler/SparkListenerBus.scala | 6 +- .../apache/spark/scheduler/StageInfo.scala | 25 +- .../org/apache/spark/scheduler/Task.scala | 1 - .../org/apache/spark/scheduler/TaskInfo.scala | 16 + .../spark/ui/jobs/JobProgressListener.scala | 32 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 11 +- .../scala/org/apache/spark/util/Utils.scala | 23 +- .../spark/scheduler/JobLoggerSuite.scala | 38 +-- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../ui/jobs/JobProgressListenerSuite.scala | 1 - 16 files changed, 336 insertions(+), 378 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 455339943f42d..85c523461aac2 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,83 +17,75 @@ package org.apache.spark.executor +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ + class TaskMetrics extends Serializable { - /** - * Host's name the task runs on - */ + /** Host's name the task runs on */ var hostname: String = _ - /** - * Time taken on the executor to deserialize this task - */ + /** Time taken on the executor to deserialize this task */ var executorDeserializeTime: Int = _ - /** - * Time the executor spends actually running the task (including fetching shuffle data) - */ + /** Time the executor spends actually running the task (including fetching shuffle data) */ var executorRunTime: Int = _ - /** - * The number of bytes this task transmitted back to the driver as the TaskResult - */ + /** The number of bytes this task transmitted back to the driver as the TaskResult */ var resultSize: Long = _ - /** - * Amount of time the JVM spent in garbage collection while executing this task - */ + /** Amount of time the JVM spent in garbage collection while executing this task */ var jvmGCTime: Long = _ - /** - * Amount of time spent serializing the task result - */ + /** Amount of time spent serializing the task result */ var resultSerializationTime: Long = _ - /** - * The number of in-memory bytes spilled by this task - */ + /** The number of in-memory bytes spilled by this task */ var memoryBytesSpilled: Long = _ - /** - * The number of on-disk bytes spilled by this task - */ + /** The number of on-disk bytes spilled by this task */ var diskBytesSpilled: Long = _ /** - * If this task reads from shuffle output, metrics on getting shuffle data will be collected here + * If this task reads from shuffle output, metrics on getting shuffle data will be + * collected here */ var shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected - * here + * If this task writes to shuffle output, metrics on the written shuffle data will be + * collected here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + def toJson: JValue = { + ("Host Name" -> hostname) ~ + ("Executor Deserialize Time" -> executorDeserializeTime) ~ + ("Executor Run Time" -> executorRunTime) ~ + ("Result Size" -> resultSize) ~ + ("JVM GC Time" -> jvmGCTime) ~ + ("Result Serialization Time" -> resultSerializationTime) ~ + ("Memory Bytes Spilled" -> memoryBytesSpilled) ~ + ("Disk Bytes Spilled" -> diskBytesSpilled) ~ + ("Shuffle Read Metrics" -> shuffleReadMetrics.map(_.toJson).getOrElse(JNothing)) ~ + ("Shuffle Write Metrics" -> shuffleWriteMetrics.map(_.toJson).getOrElse(JNothing)) + } } object TaskMetrics { private[spark] def empty(): TaskMetrics = new TaskMetrics } - class ShuffleReadMetrics extends Serializable { - /** - * Absolute time when this task finished reading shuffle data - */ + /** Absolute time when this task finished reading shuffle data */ var shuffleFinishTime: Long = _ - /** - * Number of blocks fetched in this shuffle by this task (remote or local) - */ + /** Number of blocks fetched in this shuffle by this task (remote or local) */ var totalBlocksFetched: Int = _ - /** - * Number of remote blocks fetched in this shuffle by this task - */ + /** Number of remote blocks fetched in this shuffle by this task */ var remoteBlocksFetched: Int = _ - /** - * Number of local blocks fetched in this shuffle by this task - */ + /** Number of local blocks fetched in this shuffle by this task */ var localBlocksFetched: Int = _ /** @@ -110,20 +102,29 @@ class ShuffleReadMetrics extends Serializable { */ var remoteFetchTime: Long = _ - /** - * Total number of remote bytes read from the shuffle by this task - */ + /** Total number of remote bytes read from the shuffle by this task */ var remoteBytesRead: Long = _ + + def toJson: JValue = { + ("Shuffle Finish Time" -> shuffleFinishTime) ~ + ("Total Blocks Fetched" -> totalBlocksFetched) ~ + ("Remote Blocks Fetched" -> remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> localBlocksFetched) ~ + ("Fetch Wait Time" -> fetchWaitTime) ~ + ("Remote Fetch Time" -> remoteFetchTime) ~ + ("Remote Bytes Read" -> remoteBytesRead) + } } class ShuffleWriteMetrics extends Serializable { - /** - * Number of bytes written for the shuffle by this task - */ + /** Number of bytes written for the shuffle by this task */ var shuffleBytesWritten: Long = _ - /** - * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds - */ + /** Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ var shuffleWriteTime: Long = _ + + def toJson: JValue = { + ("Shuffle Bytes Written" -> shuffleBytesWritten) ~ + ("Shuffle Write Time" -> shuffleWriteTime) + } } 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 80211541a6a63..86e86bbaa16db 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -541,13 +541,14 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job, Array(), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq(), properties)) runLocally(job) } else { idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toSeq, properties)) submitStage(finalStage) } @@ -588,13 +589,15 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) } } - listenerBus.post(SparkListenerTaskStart(task, taskInfo)) + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(task, taskInfo)) + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) + val stageId = task.stageId + val taskType = task.getClass.getSimpleName + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -612,7 +615,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, None))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) } return true } @@ -682,7 +685,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, Some(job.finalStage)) + jobResult = JobFailed(e, job.finalStage.id) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -690,7 +693,7 @@ class DAGScheduler( stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through stageToInfos -= s // completion events or stage abort jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } } @@ -838,7 +841,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) jobIdToStageIdsRemove(job.jobId) - listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -997,7 +1000,7 @@ class DAGScheduler( jobIdToStageIds -= jobId activeJobs -= job idToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1020,7 +1023,7 @@ class DAGScheduler( idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index b909b66a5de76..c4ba829f1af9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -22,23 +22,19 @@ import java.text.SimpleDateFormat import java.util.{Date, Properties} import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.{HashMap, HashSet, ListBuffer} +import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics -import org.apache.spark.storage.StorageLevel /** * A logger class to record runtime information for jobs in Spark. This class outputs one log file - * for each Spark job, containing RDD graph, tasks start/stop, shuffle information. - * JobLogger is a subclass of SparkListener, use addSparkListener to add JobLogger to a SparkContext - * after the SparkContext is created. - * Note that each JobLogger only works for one SparkContext - * @param logDirName The base directory for the log files. + * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass + * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext + * is created. Note that each JobLogger only works for one SparkContext */ -class JobLogger(val user: String, val logDirName: String) +class JobLogger(user: String, logDirName: String) extends SparkListener with Logging { def this() = this(System.getProperty("user.name", ""), @@ -51,19 +47,19 @@ class JobLogger(val user: String, val logDirName: String) "/tmp/spark-%s".format(user) } - private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIDToJobID = new HashMap[Int, Int] - private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] + private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIdToJobId = new HashMap[Int, Int] + private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents] + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() // The following 5 functions are used only in testing. private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIDtoPrintWriter = jobIDToPrintWriter - private[scheduler] def getStageIDToJobID = stageIDToJobID - private[scheduler] def getJobIDToStages = jobIDToStages + private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter + private[scheduler] def getStageIdToJobId = stageIdToJobId + private[scheduler] def getJobIdToStageIds = jobIdToStageIds private[scheduler] def getEventQueue = eventQueue /** Create a folder for log files, the folder's name is the creation time of jobLogger */ @@ -80,191 +76,90 @@ class JobLogger(val user: String, val logDirName: String) /** * Create a log file for one job - * @param jobID ID of the job + * @param jobId ID of the job * @exception FileNotFoundException Fail to create log file */ - protected def createLogWriter(jobID: Int) { + protected def createLogWriter(jobId: Int) { try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) - jobIDToPrintWriter += (jobID -> fileWriter) + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) + jobIdToPrintWriter += (jobId -> fileWriter) } catch { case e: FileNotFoundException => e.printStackTrace() } } /** - * Close log file, and clean the stage relationship in stageIDToJobID - * @param jobID ID of the job + * Close log file, and clean the stage relationship in stageIdToJobId + * @param jobId ID of the job */ - protected def closeLogWriter(jobID: Int) { - jobIDToPrintWriter.get(jobID).foreach { fileWriter => + protected def closeLogWriter(jobId: Int) { + jobIdToPrintWriter.get(jobId).foreach { fileWriter => fileWriter.close() - jobIDToStages.get(jobID).foreach(_.foreach{ stage => - stageIDToJobID -= stage.id + jobIdToStageIds.get(jobId).foreach(_.foreach{ stageId => + stageIdToJobId -= stageId }) - jobIDToPrintWriter -= jobID - jobIDToStages -= jobID + jobIdToPrintWriter -= jobId + jobIdToStageIds -= jobId } } + /** + * Build up the maps that represent stage-job relationships + * @param jobId ID of the job + * @param stageIds IDs of the associated stages + */ + protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { + jobIdToStageIds(jobId) = stageIds + stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } + } + /** * Write info into log file - * @param jobID ID of the job + * @param jobId ID of the job * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + info } - jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) } /** * Write info into log file - * @param stageID ID of the stage + * @param stageId ID of the stage * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) { - stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) - } - - /** - * Build stage dependency for a job - * @param jobID ID of the job - * @param stage Root stage of the job - */ - protected def buildJobDep(jobID: Int, stage: Stage) { - if (stage.jobId == jobID) { - jobIDToStages.get(jobID) match { - case Some(stageList) => stageList += stage - case None => val stageList = new ListBuffer[Stage] - stageList += stage - jobIDToStages += (jobID -> stageList) - } - stageIDToJobID += (stage.id -> jobID) - stage.parents.foreach(buildJobDep(jobID, _)) - } - } - - /** - * Record stage dependency and RDD dependency for a stage - * @param jobID Job ID of the stage - */ - protected def recordStageDep(jobID: Int) { - def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = { - var rddList = new ListBuffer[RDD[_]] - rddList += rdd - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - case dep: Dependency[_] => rddList ++= getRddsInStage(dep.rdd) - } - rddList - } - jobIDToStages.get(jobID).foreach {_.foreach { stage => - var depRddDesc: String = "" - getRddsInStage(stage.rdd).foreach { rdd => - depRddDesc += rdd.id + "," - } - var depStageDesc: String = "" - stage.parents.foreach { stage => - depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")" - } - jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + - depRddDesc.substring(0, depRddDesc.length - 1) + ")" + - " STAGE_DEP=" + depStageDesc, false) - } - } + protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { + stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) } /** - * Generate indents and convert to String - * @param indent Number of indents - * @return string of indents - */ - protected def indentString(indent: Int): String = { - val sb = new StringBuilder() - for (i <- 1 to indent) { - sb.append(" ") - } - sb.toString() - } - - /** - * Get RDD's name - * @param rdd Input RDD - * @return String of RDD's name - */ - protected def getRddName(rdd: RDD[_]): String = { - var rddName = rdd.getClass.getSimpleName - if (rdd.name != null) { - rddName = rdd.name - } - rddName - } - - /** - * Record RDD dependency graph in a stage - * @param jobID Job ID of the stage - * @param rdd Root RDD of the stage - * @param indent Indent number before info - */ - protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { - val rddInfo = - if (rdd.getStorageLevel != StorageLevel.NONE) { - "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " CACHED" + " " + - rdd.origin + " " + rdd.generator - } else { - "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " NONE" + " " + - rdd.origin + " " + rdd.generator - } - jobLogInfo(jobID, indentString(indent) + rddInfo, false) - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId - jobLogInfo(jobID, indentString(indent + 1) + depInfo, false) - case dep: Dependency[_] => recordRddInStageGraph(jobID, dep.rdd, indent + 1) - } - } - - /** - * Record stage dependency graph of a job - * @param jobID Job ID of the stage - * @param stage Root stage of the job - * @param indent Indent number before info, default is 0 + * Log job properties into job log file + * @param jobId ID of the job + * @param properties Properties of the job */ - protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) - { - val stageInfo = if (stage.isShuffleMap) { - "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId - } else { - "STAGE_ID=" + stage.id + " RESULT_STAGE" - } - if (stage.jobId == jobID) { - jobLogInfo(jobID, indentString(indent) + stageInfo, false) - if (!idSet.contains(stage.id)) { - idSet += stage.id - recordRddInStageGraph(jobID, stage.rdd, indent) - stage.parents.foreach(recordStageDepGraph(jobID, _, idSet, indent + 2)) - } - } else { - jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false) + protected def logJobProperties(jobId: Int, properties: Properties) { + if (properties != null) { + val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") + jobLogInfo(jobId, description, false) } } /** - * Record task metrics into job log files, including execution info and shuffle metrics - * @param stageID Stage ID of the task + * Log task metrics into job log files, including execution info and shuffle metrics + * @param stageId Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info * @param taskMetrics Task running metrics */ - protected def recordTaskMetrics(stageID: Int, status: String, + protected def logTaskMetrics(stageId: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime @@ -283,7 +178,7 @@ class JobLogger(val user: String, val logDirName: String) case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) } /** @@ -291,8 +186,9 @@ class JobLogger(val user: String, val logDirName: String) * @param stageSubmitted Stage submitted event */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - stageLogInfo(stageSubmitted.stage.stageId,"STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( - stageSubmitted.stage.stageId, stageSubmitted.stage.numTasks)) + val stageInfo = stageSubmitted.stageInfo + stageLogInfo(stageInfo.stageId, "STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( + stageInfo.stageId, stageInfo.numTasks)) } /** @@ -300,47 +196,53 @@ class JobLogger(val user: String, val logDirName: String) * @param stageCompleted Stage completed event */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format( - stageCompleted.stage.stageId)) + val stageId = stageCompleted.stageInfo.stageId + stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) } - override def onTaskStart(taskStart: SparkListenerTaskStart) { } - /** * When task ends, record task completion status and metrics * @param taskEnd Task end event */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val task = taskEnd.task val taskInfo = taskEnd.taskInfo - var taskStatus = "" - task match { - case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" - case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" - } + var taskStatus = taskEnd.taskType taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) + logTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) case Resubmitted => taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + - " STAGE_ID=" + task.stageId - stageLogInfo(task.stageId, taskStatus) + " STAGE_ID=" + taskEnd.stageId + stageLogInfo(taskEnd.stageId, taskStatus) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + - task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId - stageLogInfo(task.stageId, taskStatus) + stageLogInfo(taskEnd.stageId, taskStatus) case _ => } } + /** + * When job starts, record job property and stage graph + * @param jobStart Job start event + */ + override def onJobStart(jobStart: SparkListenerJobStart) { + val jobId = jobStart.jobId + val properties = jobStart.properties + createLogWriter(jobId) + buildJobStageDependencies(jobId, jobStart.stageIds) + logJobProperties(jobId, properties) + jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") + } + /** * When job ends, recording job completion status and close log file * @param jobEnd Job end event */ override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val job = jobEnd.job - var info = "JOB_ID=" + job.jobId + val jobId = jobEnd.jobId + var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -348,35 +250,7 @@ class JobLogger(val user: String, val logDirName: String) exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(job.jobId) - } - - /** - * Record job properties into job log file - * @param jobID ID of the job - * @param properties Properties of the job - */ - protected def recordJobProperties(jobID: Int, properties: Properties) { - if(properties != null) { - val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobID, description, false) - } - } - - /** - * When job starts, record job property and stage graph - * @param jobStart Job start event - */ - override def onJobStart(jobStart: SparkListenerJobStart) { - val job = jobStart.job - val properties = jobStart.properties - createLogWriter(job.jobId) - recordJobProperties(job.jobId, properties) - buildJobDep(job.jobId, job.finalStage) - recordStageDep(job.jobId) - recordStageDepGraph(job.jobId, job.finalStage, new HashSet[Int]) - jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED") + jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(jobId) } } - diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index d94f6ad924260..9f4643738f970 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,11 +17,25 @@ package org.apache.spark.scheduler +import net.liftweb.json.JsonAST._ +import net.liftweb.json.JsonDSL._ + /** * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +private[spark] sealed trait JobResult { + def toJson: JValue +} + +private[spark] case object JobSucceeded extends JobResult { + def toJson = ("Status" -> "Success") +} -private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) - extends JobResult +private[spark] case class JobFailed(exception: Exception, failedStageId: Int) + extends JobResult { + def toJson = { + ("Status" -> "Failed") ~ + ("Exception" -> exception.getMessage) ~ + ("Failed Stage ID" -> failedStageId) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index b026f860a8cd8..8007b5418741e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, None) + jobResult = JobFailed(exception, -1) this.notifyAll() } 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 129153c732d9a..971535117c149 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -18,51 +18,105 @@ package org.apache.spark.scheduler import java.util.Properties + import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics -sealed trait SparkListenerEvents +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ + +sealed trait SparkListenerEvent { + def toJson: JValue +} -case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties) - extends SparkListenerEvents +case class SparkListenerStageSubmitted( + stageInfo: StageInfo, + properties: Properties) + extends SparkListenerEvent { + def toJson = { + ("Event" -> "Stage Submitted") ~ + ("Stage Info" -> stageInfo.toJson) ~ + ("Properties" -> Utils.propertiesToJson(properties)) + } +} -case class SparkListenerStageCompleted(stage: StageInfo) extends SparkListenerEvents +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent { + def toJson = { + ("Event" -> "Stage Completed") ~ + ("Stage Info" -> stageInfo.toJson) + } +} -case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent { + def toJson = { + ("Event" -> "Task Start") ~ + ("Stage ID" -> stageId) ~ + ("Task Info" -> taskInfo.toJson) + } +} -case class SparkListenerTaskGettingResult( - task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent { + def toJson = { + ("Event" -> "Task Getting Result") ~ + ("Task Info" -> taskInfo.toJson) + } +} -case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, - taskMetrics: TaskMetrics) extends SparkListenerEvents +case class SparkListenerTaskEnd( + stageId: Int, + taskType: String, + reason: TaskEndReason, + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) + extends SparkListenerEvent { + def toJson = { + ("Event" -> "Task End") ~ + ("Stage ID" -> stageId) ~ + ("Task Type" -> taskType) ~ + ("Task End Reason" -> reason.toString) ~ + ("Task Info" -> taskInfo.toJson) ~ + ("Task Metrics" -> taskMetrics.toJson) + } +} -case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], - properties: Properties = null) extends SparkListenerEvents +case class SparkListenerJobStart( + jobId: Int, + stageIds: Seq[Int], + properties: Properties) + extends SparkListenerEvent { + def toJson = { + ("Event" -> "Job Start") ~ + ("Job ID" -> jobId) ~ + ("Stage IDs" -> JArray(stageIds.map(JInt(_)).toList)) ~ + ("Properties" -> Utils.propertiesToJson(properties)) + } +} -case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) - extends SparkListenerEvents +case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent { + def toJson = { + ("Event" -> "Job End") ~ + ("Job ID" -> jobId) ~ + ("Job Result" -> jobResult.toJson) + } +} /** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvents +private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent { + def toJson = ("Event" -> "Shutdown") +} /** * Interface for listening to events from the Spark scheduler. */ trait SparkListener { - /** - * Called when a stage is completed, with information on the completed stage - */ + /** Called when a stage is completed, with information on the completed stage */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } - /** - * Called when a stage is submitted - */ + /** Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - /** - * Called when a task starts - */ + /** Called when a task starts */ def onTaskStart(taskStart: SparkListenerTaskStart) { } /** @@ -71,21 +125,14 @@ trait SparkListener { */ def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } - /** - * Called when a task ends - */ + /** Called when a task ends */ def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } - /** - * Called when a job starts - */ + /** Called when a job starts */ def onJobStart(jobStart: SparkListenerJobStart) { } - /** - * Called when a job ends - */ + /** Called when a job ends */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } - } /** @@ -95,7 +142,7 @@ class StatsReportListener extends SparkListener with Logging { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted - this.logInfo("Finished stage: " + stageCompleted.stage) + this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) //shuffle write @@ -111,7 +158,7 @@ class StatsReportListener extends SparkListener with Logging { //runtime breakdown - val runtimePcts = stageCompleted.stage.taskInfos.map{ + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", @@ -131,15 +178,15 @@ private[spark] object StatsReportListener extends Logging { val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" def extractDoubleDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Double]) + getMetric: (TaskInfo, TaskMetrics) => Option[Double]) : Option[Distribution] = { - Distribution(stage.stage.taskInfos.flatMap { + Distribution(stage.stageInfo.taskInfos.flatMap { case ((info,metric)) => getMetric(info, metric)}) } //is there some way to setup the types that I can get rid of this completely? def extractLongDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Long]) + getMetric: (TaskInfo, TaskMetrics) => Option[Long]) : Option[Distribution] = { extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) } @@ -170,7 +217,7 @@ private[spark] object StatsReportListener extends Logging { showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) } - def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) + def showBytesDistribution(heading:String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) (implicit stage: SparkListenerStageCompleted) { showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 17b1328b86788..b46e87f87a450 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -23,14 +23,14 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.Logging -/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ +/** Asynchronously passes SparkListenerEvent's to registered SparkListeners. */ private[spark] class SparkListenerBus extends Logging { private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener] /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents](EVENT_QUEUE_CAPACITY) + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false // Create a new daemon thread to listen for events. This thread is stopped when it receives @@ -68,7 +68,7 @@ private[spark] class SparkListenerBus extends Logging { sparkListeners += listener } - def post(event: SparkListenerEvents) { + def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) if (!eventAdded && !queueFullErrorMessageLogged) { logError("Dropping SparkListenerEvent because no remaining room in event queue. " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8f320e5c7a74b..71f02c1ee008b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -21,18 +21,18 @@ import scala.collection._ import org.apache.spark.executor.TaskMetrics +import net.liftweb.json.JsonAST._ +import net.liftweb.json.JsonDSL._ + /** * Stores information about a stage to pass from the scheduler to SparkListeners. * * taskInfos stores the metrics for all tasks that have completed, including redundant, speculated * tasks. */ -class StageInfo( - stage: Stage, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = - mutable.Buffer[(TaskInfo, TaskMetrics)]() -) { +class StageInfo(stage: Stage) { val stageId = stage.id + val taskInfos = mutable.Buffer[(TaskInfo, TaskMetrics)]() /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None @@ -41,4 +41,19 @@ class StageInfo( val numPartitions = stage.numPartitions val numTasks = stage.numTasks var emittedTaskSizeWarning = false + + def toJson: JValue = { + val (taskInfoList, taskMetricsList) = taskInfos.toList.unzip + val taskInfoJson = JArray(taskInfoList.map(_.toJson)) + val taskMetricsJson = JArray(taskMetricsList.map(_.toJson)) + ("Stage ID" -> stage.id) ~ + ("Submission Time" -> submissionTime.getOrElse(0L)) ~ + ("Completion Time" -> completionTime.getOrElse(0L)) ~ + ("RDD Name" -> rddName) ~ + ("Stage Name" -> name) ~ + ("Number of Partitions" -> numPartitions) ~ + ("Number of Tasks" -> numTasks) ~ + ("Task Info" -> taskInfoJson) ~ + ("Task Metrics" -> taskMetricsJson) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 69b42e86eae3e..b85b4a50cd93a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -29,7 +29,6 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream - /** * A unit of execution. We have two kinds of Task's in Spark: * - [[org.apache.spark.scheduler.ShuffleMapTask]] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6183b125def99..aa4e73ff0c804 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,6 +17,9 @@ package org.apache.spark.scheduler +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ + /** * Information about a running task attempt inside a TaskSet. */ @@ -90,4 +93,17 @@ class TaskInfo( } def timeRunning(currentTime: Long): Long = currentTime - launchTime + + def toJson: JValue = { + ("Task ID" -> taskId) ~ + ("Index" -> index) ~ + ("Launch Time" -> launchTime) ~ + ("Executor ID" -> executorId) ~ + ("Host" -> host) ~ + ("Locality" -> taskLocality.toString) ~ + ("Status" -> status) ~ + ("Getting Result Time" -> gettingResultTime) ~ + ("Finish Time" -> finishTime) ~ + ("Failed" -> failed) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 07a08f5277d19..b9a02c576fbfb 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -63,7 +63,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onJobStart(jobStart: SparkListenerJobStart) {} override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { - val stage = stageCompleted.stage + val stage = stageCompleted.stageInfo poolToActiveStages(stageIdToPool(stage.stageId)) -= stage activeStages -= stage completedStages += stage @@ -93,7 +93,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val stage = stageSubmitted.stage + val stage = stageSubmitted.stageInfo activeStages += stage val poolName = Option(stageSubmitted.properties).map { @@ -111,7 +111,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val sid = taskStart.task.stageId + val sid = taskStart.stageId val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) tasksActive += taskStart.taskInfo val taskList = stageIdToTaskInfos.getOrElse( @@ -127,7 +127,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val sid = taskEnd.task.stageId + val sid = taskEnd.stageId // create executor summary map if necessary val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, @@ -205,20 +205,16 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - /* If two jobs share a stage we could get this failure message twice. So we first - * check whether we've already retired this stage. */ - val stageInfo = activeStages.filter(s => s.stageId == stage.id).headOption - stageInfo.foreach {s => - activeStages -= s - poolToActiveStages(stageIdToPool(stage.id)) -= s - failedStages += s - trimIfNecessary(failedStages) - } - case _ => + jobEnd.jobResult match { + case JobFailed(_, stageId) => + // If two jobs share a stage we could get this failure message twice. + // So we first check whether we've already retired this stage. + val stageInfo = activeStages.filter(s => s.stageId == stageId).headOption + stageInfo.foreach {s => + activeStages -= s + poolToActiveStages(stageIdToPool(stageId)) -= s + failedStages += s + trimIfNecessary(failedStages) } case _ => } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index c1ee2f3d00d66..a0be27abad917 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -17,23 +17,14 @@ package org.apache.spark.ui.jobs -import scala.concurrent.duration._ - import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import scala.Seq -import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} - import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.{ExceptionFailure, SparkContext, Success} -import org.apache.spark.scheduler._ -import collection.mutable -import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.SparkContext import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8749ab7875bb7..5ba05501fc154 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -19,10 +19,12 @@ package org.apache.spark.util import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} -import java.util.{Locale, Random, UUID} +import java.nio.ByteBuffer +import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source @@ -31,15 +33,13 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} -import org.apache.hadoop.io._ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil -import java.nio.ByteBuffer import org.apache.spark.{SparkConf, SparkException, Logging} +import net.liftweb.json.JsonAST._ /** * Various utility methods used by Spark. @@ -868,4 +868,19 @@ private[spark] object Utils extends Logging { } count } + + /** Convert a (String, String) map to a json object */ + def mapToJson(m: Map[String, String]): JValue = { + val jsonFields = m.map { case (k, v) => + JField(k, JString(v)) + } + JObject(jsonFields.toList) + } + + /** Convert a java Properties to a json object */ + def propertiesToJson(properties: Properties): JValue = { + Option(properties).map { p => + Utils.mapToJson(p.asScala) + }.getOrElse(JNothing) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 29102913c719c..6b9e4b3a71e53 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -17,11 +17,6 @@ package org.apache.spark.scheduler -import java.util.Properties -import java.util.concurrent.LinkedBlockingQueue - -import scala.collection.mutable - import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers @@ -29,7 +24,6 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD - class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { val WAIT_TIMEOUT_MILLIS = 10000 @@ -38,8 +32,6 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val joblogger = new JobLogger { def createLogWriterTest(jobID: Int) = createLogWriter(jobID) def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) - def getRddNameTest(rdd: RDD[_]) = getRddName(rdd) - def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) } type MyRDD = RDD[(Int, Int)] def makeRdd(numPartitions: Int, dependencies: List[Dependency[_]]): MyRDD = { @@ -63,26 +55,22 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rootStageInfo = new StageInfo(rootStage) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) - joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getSimpleName) - parentRdd.setName("MyRDD") - joblogger.getRddNameTest(parentRdd) should be ("MyRDD") joblogger.createLogWriterTest(jobID) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.buildJobDepTest(jobID, rootStage) - joblogger.getJobIDToStages.get(jobID).get.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) - joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) + joblogger.getJobIdToPrintWriter.size should be (1) + joblogger.getJobIdToStageIds.get(jobID).get.size should be (2) + joblogger.getStageIdToJobId.get(0) should be (Some(jobID)) + joblogger.getStageIdToJobId.get(1) should be (Some(jobID)) joblogger.closeLogWriterTest(jobID) - joblogger.getStageIDToJobID.size should be (0) - joblogger.getJobIDToStages.size should be (0) - joblogger.getJobIDtoPrintWriter.size should be (0) + joblogger.getStageIdToJobId.size should be (0) + joblogger.getJobIdToStageIds.size should be (0) + joblogger.getJobIdToPrintWriter.size should be (0) } test("inner variables") { sc = new SparkContext("local[4]", "joblogger") val joblogger = new JobLogger { override protected def closeLogWriter(jobID: Int) = - getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => + getJobIdToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() } } @@ -95,11 +83,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.getStageIDToJobID.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(0)) - joblogger.getStageIDToJobID.get(1) should be (Some(0)) - joblogger.getJobIDToStages.size should be (1) + joblogger.getJobIdToPrintWriter.size should be (1) + joblogger.getStageIdToJobId.size should be (2) + joblogger.getStageIdToJobId.get(0) should be (Some(0)) + joblogger.getStageIdToJobId.get(1) should be (Some(0)) + joblogger.getJobIdToStageIds.size should be (1) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 368c5154ea3b9..4979d7dd6f786 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -208,7 +208,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc class SaveStageInfo extends SparkListener { val stageInfos = Buffer[StageInfo]() override def onStageCompleted(stage: SparkListenerStageCompleted) { - stageInfos += stage.stage + stageInfos += stage.stageInfo } } 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 67a57a0e7f9d0..524c80d1520e6 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 @@ -20,7 +20,6 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.apache.spark.scheduler._ import org.apache.spark.{LocalSparkContext, SparkContext, Success} -import org.apache.spark.scheduler.SparkListenerTaskStart import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} class JobProgressListenerSuite extends FunSuite with LocalSparkContext { From bb222b9f7422cdf9e3a4c682bb271da1f75f4f75 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 12 Feb 2014 20:35:09 -0800 Subject: [PATCH 10/68] ExecutorUI: render completely from JSON Additionally, this commit fixes the bug in the local mode, where executor IDs of tasks do not match those of storage statuses (more detail in ExecutorsUI.scala). This commit currently does not serialize the SparkListenerEvents yet, but instead serializes changes to each executor JSON. This is a big TODO in the upcoming commit. --- .../spark/scheduler/SparkListener.scala | 19 +- .../apache/spark/scheduler/StageInfo.scala | 4 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 276 +++++++++++------- .../org/apache/spark/util/FileLogger.scala | 65 +++-- .../scala/org/apache/spark/util/Utils.scala | 41 ++- 5 files changed, 268 insertions(+), 137 deletions(-) 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 971535117c149..81409431f4375 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -145,21 +145,20 @@ class StatsReportListener extends SparkListener with Logging { this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) - //shuffle write + // Shuffle write showBytesDistribution("shuffle bytes written:", (_,metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten)) - //fetch & io + // Fetch & I/O showMillisDistribution("fetch wait time:", (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime)) showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead)) showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) - //runtime breakdown - - val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ - case (info, metrics) => RuntimePercentage(info.duration, metrics) + // Runtime breakdown + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => + RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") @@ -172,7 +171,7 @@ class StatsReportListener extends SparkListener with Logging { private[spark] object StatsReportListener extends Logging { - //for profiling, the extremes are more interesting + // For profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) val probabilities = percentiles.map{_ / 100.0} val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" @@ -184,7 +183,7 @@ private[spark] object StatsReportListener extends Logging { case ((info,metric)) => getMetric(info, metric)}) } - //is there some way to setup the types that I can get rid of this completely? + // Is there some way to setup the types that I can get rid of this completely? def extractLongDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) : Option[Distribution] = { @@ -244,9 +243,7 @@ private[spark] object StatsReportListener extends Logging { val minutes = seconds * 60 val hours = minutes * 60 - /** - * reformat a time interval in milliseconds to a prettier format for output - */ + /** Reformat a time interval in milliseconds to a prettier format for output */ def millisToString(ms: Long) = { val (size, units) = if (ms > hours) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 71f02c1ee008b..325d9b17c0fbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -26,12 +26,10 @@ import net.liftweb.json.JsonDSL._ /** * Stores information about a stage to pass from the scheduler to SparkListeners. - * - * taskInfos stores the metrics for all tasks that have completed, including redundant, speculated - * tasks. */ class StageInfo(stage: Stage) { val stageId = stage.id + /** Store the metrics for all tasks that have completed, including redundant, speculated tasks. */ val taskInfos = mutable.Buffer[(TaskInfo, TaskMetrics)]() /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 3f5ce2f68eba5..9ccf536a2a6ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -25,25 +25,22 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler import org.apache.spark.{ExceptionFailure, Logging, SparkContext} -import org.apache.spark.scheduler.{SparkListenerJobEnd, SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} +import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils import org.apache.spark.util.{Utils, FileLogger} -import net.liftweb.json.DefaultFormats import net.liftweb.json.JsonAST._ import net.liftweb.json.JsonDSL._ private[spark] class ExecutorsUI(val sc: SparkContext) { private var _listener: Option[ExecutorsListener] = None - private implicit val format = DefaultFormats - def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener) + _listener = Some(new ExecutorsListener(sc)) sc.addSparkListener(listener) } @@ -51,49 +48,30 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { ("/executors", (request: HttpServletRequest) => render(request)) ) + /** Render an HTML page that encodes executor information */ def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - - val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) - - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read", - "Shuffle Write") - - def execRow(kv: Seq[String]) = { - - {kv(0)} - {kv(1)} - {kv(2)} - - {Utils.bytesToString(kv(3).toLong)} / {Utils.bytesToString(kv(4).toLong)} - - - {Utils.bytesToString(kv(5).toLong)} - - {kv(6)} - {kv(7)} - {kv(8)} - {kv(9)} - {Utils.msDurationToString(kv(10).toLong)} - {Utils.bytesToString(kv(11).toLong)} - {Utils.bytesToString(kv(12).toLong)} - - } - - val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) - val execTable = UIUtils.listingTable(execHead, execRow, execInfo) + listener.updateStorageStatusFromEnv() + val summaryJson = listener.summaryJson + val executorsJson = listener.executorIdToJson.values.toSeq + renderFromJson(summaryJson, executorsJson) + } + /** Render an HTML page that encodes executor information from the given JSON representations */ + def renderFromJson(summaryJson: JValue, executorsJson: Seq[JValue]): Seq[Node] = { + val memoryAvailable = Utils.extractLongFromJson(summaryJson, "Memory Available").getOrElse(0L) + val memoryUsed = Utils.extractLongFromJson(summaryJson, "Memory Used").getOrElse(0L) + val diskSpaceUsed = Utils.extractLongFromJson(summaryJson, "Disk Space Used").getOrElse(0L) + val execTable = UIUtils.listingTable[JValue](execHeader, execRow, executorsJson) val content =
    -
  • Memory: - {Utils.bytesToString(memUsed)} Used - ({Utils.bytesToString(maxMem)} Total)
  • -
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
  • +
  • + Memory: + {Utils.bytesToString(memoryAvailable)} Used + ({Utils.bytesToString(memoryUsed)} Total) +
  • +
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
@@ -103,48 +81,77 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { ; - UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, sc, "Executors (" + executorsJson.size + ")", Executors) } - def getExecInfo(statusId: Int): Seq[String] = { - val status = sc.getExecutorStorageStatus(statusId) - val execId = status.blockManagerId.executorId - val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size.toString - val memUsed = status.memUsed().toString - val maxMem = status.maxMem.toString - val diskUsed = status.diskUsed().toString - val activeTasks = listener.getJson(execId, "Active Tasks").extract[Int] - val failedTasks = listener.getJson(execId, "Failed Tasks").extract[Int] - val completeTasks = listener.getJson(execId, "Complete Tasks").extract[Int] + /** Header fields in the executors table */ + private def execHeader = Seq( + "Executor ID", + "Address", + "RDD Blocks", + "Memory Used", + "Disk Used", + "Active Tasks", + "Failed Tasks", + "Complete Tasks", + "Total Tasks", + "Task Time", + "Shuffle Read", + "Shuffle Write") + + /** Render an HTML table row representing an executor from the given JSON representation */ + private def execRow(executorJson: JValue): Seq[Node] = { + def getString(field: String) = Utils.extractStringFromJson(executorJson, field).getOrElse("") + def getLong(field: String) = Utils.extractLongFromJson(executorJson, field).getOrElse(0L) + def getInt(field: String) = Utils.extractIntFromJson(executorJson, field).getOrElse(0) + val memoryUsed = getLong("Memory Used") + val memoryAvailable = getLong("Memory Available") + val diskUsed = getLong("Disk Used") + val activeTasks = getInt("Active Tasks") + val failedTasks = getInt("Failed Tasks") + val completeTasks = getInt("Complete Tasks") val totalTasks = activeTasks + failedTasks + completeTasks - val totalDuration = listener.getJson(execId, "Task Time").extract[Long] - val totalShuffleRead = listener.getJson(execId, "Shuffle Read").extract[Long] - val totalShuffleWrite = listener.getJson(execId, "Shuffle Write").extract[Long] - - Seq( - execId, - hostPort, - rddBlocks, - memUsed, - maxMem, - diskUsed, - activeTasks.toString, - failedTasks.toString, - completeTasks.toString, - totalTasks.toString, - totalDuration.toString, - totalShuffleRead.toString, - totalShuffleWrite.toString - ) + + + {getString("Executor ID")} + {getString("Address")} + {getString("RDD Blocks")} + + {Utils.bytesToString(memoryUsed)} / + {Utils.bytesToString(memoryAvailable)} + + + {Utils.bytesToString(diskUsed)} + + {activeTasks} + {failedTasks} + {completeTasks} + {totalTasks} + {Utils.msDurationToString(getLong("Task Time"))} + {Utils.bytesToString(getLong("Shuffle Read"))} + {Utils.bytesToString(getLong("Shuffle Write"))} + } - private[spark] class ExecutorsListener extends SparkListener with Logging { + /** + * A SparkListener that maintains and logs information to be displayed on the Executors UI. + * + * Both intermediate data that resides in memory and persisted data that resides on disk are + * in JSON format. + */ + private[spark] class ExecutorsListener(sc: SparkContext) extends SparkListener with Logging { + var summaryJson: JValue = JNothing val executorIdToJson = HashMap[String, JValue]() - val logger = new FileLogger("executors-ui") + private val logger = new FileLogger("executors-ui") - def newJson(execId: String): JValue = { + /** Return the JSON representation of a newly discovered executor */ + private def newExecutorJson(execId: String): JValue = { ("Executor ID" -> execId) ~ + ("Address" -> "") ~ + ("RDD Blocks" -> "") ~ + ("Memory Used" -> 0L) ~ + ("Memory Available" -> 0L) ~ + ("Disk Used" -> 0L) ~ ("Active Tasks" -> 0) ~ ("Failed Tasks" -> 0) ~ ("Complete Tasks" -> 0) ~ @@ -153,52 +160,117 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { ("Shuffle Write" -> 0L) } - def getJson(execId: String, field: String): JValue = { - executorIdToJson.get(execId) match { - case Some(json) => (json \ field) - case None => JNothing + /** + * Update the summary and per-executor storage status from SparkEnv. This involves querying + * the driver and waiting for a reply, and so should be called sparingly. + */ + def updateStorageStatusFromEnv() { + + // Update summary storage information + val storageStatusList = sc.getExecutorStorageStatus + val memoryAvailable = storageStatusList.map(_.maxMem).fold(0L)(_+_) + val memoryUsed = storageStatusList.map(_.memUsed).fold(0L)(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_+_) + + summaryJson = + ("Memory Available" -> memoryAvailable) ~ + ("Memory Used" -> memoryUsed) ~ + ("Disk Space Used" -> diskSpaceUsed) + + // Update storage status for each executor + storageStatusList.foreach { status => + val execId = status.blockManagerId.executorId + val address = status.blockManagerId.hostPort + val rddBlocks = status.blocks.size + val memoryUsed = status.memUsed + val memoryAvailable = status.maxMem + val diskUsed = status.diskUsed + val json = executorIdToJson.getOrElse(execId, newExecutorJson(execId)) + executorIdToJson(execId) = json.transform { + case JField("Address", _) => JField("Address", JString(address)) + case JField("RDD Blocks", _) => JField("RDD Blocks", JInt(rddBlocks)) + case JField("Memory Used", _) => JField("Memory Used", JInt(memoryUsed)) + case JField("Memory Available", _) => JField("Memory Available", JInt(memoryAvailable)) + case JField("Disk Used", _) => JField("Disk Used", JInt(diskUsed)) + } + logJson(executorIdToJson(execId)) } } - def logJson(json: JValue) = logger.logLine(compactRender(json)) + override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() + + override def onStageSubmitted(stageStart: SparkListenerStageSubmitted) = { + updateStorageStatusFromEnv() + logger.flush() + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { + updateStorageStatusFromEnv() + logger.flush() + } override def onTaskStart(taskStart: SparkListenerTaskStart) = { - val eid = taskStart.taskInfo.executorId - var json = executorIdToJson.getOrElseUpdate(eid, newJson(eid)) - json = json.transform { + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). This results in + * duplicate rows for the same executor. Thus, in this mode, we aggregate these two + * rows and use the executor ID of "" to be consistent. + */ + val execId = if (sc.isLocal) "" else taskStart.taskInfo.executorId + val json = executorIdToJson.getOrElse(execId, { + // The executor ID according to the task is different from that according to SparkEnv + // This should never happen under normal circumstances... + logWarning("New executor detected during task start (%s)".format(execId)) + newExecutorJson(execId) + }) + executorIdToJson(execId) = json.transform { case JField("Active Tasks", JInt(s)) => JField("Active Tasks", JInt(s + 1)) } - executorIdToJson(eid) = json - logJson(json) + logJson(executorIdToJson(execId)) } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = { - val eid = taskEnd.taskInfo.executorId - val exception = taskEnd.reason match { - case _: ExceptionFailure => true - case _ => false - } + val execId = if (sc.isLocal) "" else taskEnd.taskInfo.executorId val newDuration = taskEnd.taskInfo.duration - var newShuffleRead = 0 - var newShuffleWrite = 0 + var newShuffleRead = 0L + var newShuffleWrite = 0L if (taskEnd.taskMetrics != null) { taskEnd.taskMetrics.shuffleReadMetrics.foreach(newShuffleRead += _.remoteBytesRead) taskEnd.taskMetrics.shuffleWriteMetrics.foreach(newShuffleWrite += _.shuffleBytesWritten) } - var json = executorIdToJson.getOrElseUpdate(eid, newJson(eid)) - json = json.transform { - case JField("Active Tasks", JInt(s)) if s > 0 => JField("Active Tasks", JInt(s - 1)) - case JField("Failed Tasks", JInt(s)) if exception => JField("Failed Tasks", JInt(s + 1)) - case JField("Complete Tasks", JInt(s)) if !exception => - JField("Complete Tasks", JInt(s + 1)) + val success = taskEnd.reason match { + case _: ExceptionFailure => false + case _ => true + } + val json = executorIdToJson.getOrElse(execId, { + // Information for this executor has vanished over the course of the task execution + // This should never happen under normal circumstances... + logWarning("New executor detected during task end (%s)".format(execId)) + newExecutorJson(execId) + }) + executorIdToJson(execId) = json.transform { + case JField("Active Tasks", JInt(t)) if t > 0 => JField("Active Tasks", JInt(t - 1)) + case JField("Failed Tasks", JInt(t)) if !success => JField("Failed Tasks", JInt(t + 1)) + case JField("Complete Tasks", JInt(t)) if success => JField("Complete Tasks", JInt(t + 1)) case JField("Task Time", JInt(s)) => JField("Task Time", JInt(s + newDuration)) case JField("Shuffle Read", JInt(s)) => JField("Shuffle Read", JInt(s + newShuffleRead)) case JField("Shuffle Write", JInt(s)) => JField("Shuffle Write", JInt(s + newShuffleWrite)) } - executorIdToJson(eid) = json - logJson(json) + logJson(executorIdToJson(execId)) } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() + /** Log summary storage status **/ + def logSummary() = logJson(summaryJson) + + /** Log storage status for the executor with the given ID */ + def logExecutor(execId: String) = logJson(executorIdToJson.getOrElse(execId, JNothing)) + + private def logJson(json: JValue) = { + if (json != JNothing) { + logger.logLine(compactRender(json)) + } + } } } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 2afc49f2e4330..c1d179ba1d687 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,20 +17,22 @@ package org.apache.spark.util -import java.io.{IOException, File, PrintWriter} +import java.io._ import java.text.SimpleDateFormat import java.util.Date -import org.apache.spark._ - /** * A generic class for logging information to file + * @param user User identifier if SPARK_LOG_DIR is not set, in which case log directory + * defaults to /tmp/spark-[user] + * @param name Name of logger, also the name of the log file + * @param flushFrequency How many writes until the results are flushed to disk */ - -class FileLogger(user: String, name: String, flushFrequency: Int = 1) extends Logging { +class FileLogger(user: String, name: String, flushFrequency: Int = 100) { private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private var logCount = 0 + private var fileIndex = 0 private val logDir = if (System.getenv("SPARK_LOG_DIR") != null) { @@ -39,11 +41,11 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 1) extends Lo "/tmp/spark-%s".format(user) } - private val logFile = logDir + "/" + name + private val logFileBase = logDir + "/" + name - private val writer: PrintWriter = { + private var writer: Option[PrintWriter] = { createLogDir() - new PrintWriter(logFile) + Some(createWriter()) // Overwrite any existing file } def this() = this(System.getProperty("user.name", ""), @@ -52,38 +54,63 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 1) extends Lo def this(_name: String) = this(System.getProperty("user.name", ""), _name) /** Create a logging directory with the given path */ - private def createLogDir() { + private def createLogDir() = { val dir = new File(logDir) if (!dir.exists && !dir.mkdirs()) { // Logger should throw a exception rather than continue to construct this object throw new IOException("create log directory error:" + logDir) } - val file = new File(logFile) - if (file.exists) { - logWarning("Overwriting existing log file at %s".format(logFile)) - } } - /** Log the message to the given writer if it exists, optionally including the time */ + /** Create a new writer to the file identified with the given path */ + private def createWriter() = { + val fileWriter = new FileWriter(logFileBase + "-" + fileIndex) + val bufferedWriter = new BufferedWriter(fileWriter) + new PrintWriter(bufferedWriter) + } + + /** + * Log the message to the given writer + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp + */ def log(msg: String, withTime: Boolean = false) = { var writeInfo = msg if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + msg } - writer.print(writeInfo) + writer.foreach(_.print(writeInfo)) logCount += 1 if (logCount % flushFrequency == 0) { - writer.flush() + flush() logCount = 0 } } /** - * Log the message as a new line to the given writer if it exists, optionally including the time + * Log the message to the given writer as a new line + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp */ def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) - /** Close the writer, if it exists */ - def close() = writer.close() + /** Flush the writer to disk manually */ + def flush() = writer.foreach(_.flush()) + + /** Close the writer. Any subsequent calls to log or flush will have no effect. */ + def close() = { + writer.foreach(_.close()) + writer = None + } + + /** Start a new writer (for a new file) if there does not already exist one */ + def start() = { + writer match { + case Some(w) => + case None => + fileIndex += 1 + writer = Some(createWriter()) + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 5ba05501fc154..8086b8c7a341f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -40,12 +40,16 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.{SparkConf, SparkException, Logging} import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { + /** For extracting fields from JSON objects */ + implicit val format = DefaultFormats + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -869,7 +873,7 @@ private[spark] object Utils extends Logging { count } - /** Convert a (String, String) map to a json object */ + /** Convert a (String, String) map to a JSON object */ def mapToJson(m: Map[String, String]): JValue = { val jsonFields = m.map { case (k, v) => JField(k, JString(v)) @@ -877,10 +881,43 @@ private[spark] object Utils extends Logging { JObject(jsonFields.toList) } - /** Convert a java Properties to a json object */ + /** Convert a java Properties to a JSON object */ def propertiesToJson(properties: Properties): JValue = { Option(properties).map { p => Utils.mapToJson(p.asScala) }.getOrElse(JNothing) } + + /** Extract a field from the given JSON AST as a scala option */ + def extractFromJson(json: JValue, field: String): Option[JValue] = { + (json \ field) match { + case JNothing => None + case value => Some(value) + } + } + + /** Extracts a string from the given JSON AST */ + def extractStringFromJson(json: JValue, field: String): Option[String] = { + extractFromJson(json, field).map(_.extract[String]) + } + + /** Extracts a double from the given JSON AST */ + def extractDoubleFromJson(json: JValue, field: String): Option[Double] = { + extractFromJson(json, field).map(_.extract[Double]) + } + + /** Extracts a long from the given JSON AST */ + def extractLongFromJson(json: JValue, field: String): Option[Long] = { + extractFromJson(json, field).map(_.extract[Long]) + } + + /** Extracts an int from the given JSON AST */ + def extractIntFromJson(json: JValue, field: String): Option[Int] = { + extractFromJson(json, field).map(_.extract[Int]) + } + + /** Extracts a boolean from the given JSON AST */ + def extractBooleanFromJson(json: JValue, field: String): Option[Boolean] = { + extractFromJson(json, field).map(_.extract[Boolean]) + } } From bf0b2e9e92d760d49ba7b26aaa41b9e3aef2420f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 13 Feb 2014 19:12:53 -0800 Subject: [PATCH 11/68] ExecutorUI: Serialize events rather than arbitary executor information This involves adding a new SparkListenerStorageFetchEvent, and adding JSON serializability to all of the objects it depends on. --- .../apache/spark/executor/TaskMetrics.scala | 77 ++++-- .../apache/spark/scheduler/JobResult.scala | 9 +- .../spark/scheduler/SparkListener.scala | 73 +++-- .../apache/spark/scheduler/StageInfo.scala | 15 +- .../org/apache/spark/scheduler/TaskInfo.scala | 8 +- .../org/apache/spark/storage/BlockId.scala | 7 +- .../apache/spark/storage/BlockManagerId.scala | 15 +- .../storage/BlockManagerMasterActor.scala | 11 +- .../apache/spark/storage/StorageLevel.scala | 12 +- .../apache/spark/storage/StorageUtils.scala | 22 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 251 +++++++++++------- .../org/apache/spark/util/FileLogger.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 41 ++- 13 files changed, 359 insertions(+), 184 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 85c523461aac2..282b7e5dbee07 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,37 +17,54 @@ package org.apache.spark.executor +import org.apache.spark.scheduler.JsonSerializable + import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ -class TaskMetrics extends Serializable { - /** Host's name the task runs on */ +class TaskMetrics extends Serializable with JsonSerializable { + /** + * Host's name the task runs on + */ var hostname: String = _ - /** Time taken on the executor to deserialize this task */ + /** + * Time taken on the executor to deserialize this task + */ var executorDeserializeTime: Int = _ - /** Time the executor spends actually running the task (including fetching shuffle data) */ + /** + * Time the executor spends actually running the task (including fetching shuffle data) + */ var executorRunTime: Int = _ - /** The number of bytes this task transmitted back to the driver as the TaskResult */ + /** + * The number of bytes this task transmitted back to the driver as the TaskResult + */ var resultSize: Long = _ - /** Amount of time the JVM spent in garbage collection while executing this task */ + /** + * Amount of time the JVM spent in garbage collection while executing this task + */ var jvmGCTime: Long = _ - /** Amount of time spent serializing the task result */ + /** + * Amount of time spent serializing the task result + */ var resultSerializationTime: Long = _ - /** The number of in-memory bytes spilled by this task */ + /** + * The number of in-memory bytes spilled by this task + */ var memoryBytesSpilled: Long = _ - /** The number of on-disk bytes spilled by this task */ + /** + * The number of on-disk bytes spilled by this task + */ var diskBytesSpilled: Long = _ /** - * If this task reads from shuffle output, metrics on getting shuffle data will be - * collected here + * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ var shuffleReadMetrics: Option[ShuffleReadMetrics] = None @@ -57,7 +74,7 @@ class TaskMetrics extends Serializable { */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None - def toJson: JValue = { + override def toJson = { ("Host Name" -> hostname) ~ ("Executor Deserialize Time" -> executorDeserializeTime) ~ ("Executor Run Time" -> executorRunTime) ~ @@ -75,17 +92,25 @@ object TaskMetrics { private[spark] def empty(): TaskMetrics = new TaskMetrics } -class ShuffleReadMetrics extends Serializable { - /** Absolute time when this task finished reading shuffle data */ +class ShuffleReadMetrics extends Serializable with JsonSerializable { + /** + * Absolute time when this task finished reading shuffle data + */ var shuffleFinishTime: Long = _ - /** Number of blocks fetched in this shuffle by this task (remote or local) */ + /** + * Number of blocks fetched in this shuffle by this task (remote or local) + */ var totalBlocksFetched: Int = _ - /** Number of remote blocks fetched in this shuffle by this task */ + /** + * Number of remote blocks fetched in this shuffle by this task + */ var remoteBlocksFetched: Int = _ - /** Number of local blocks fetched in this shuffle by this task */ + /** + * Number of local blocks fetched in this shuffle by this task + */ var localBlocksFetched: Int = _ /** @@ -102,10 +127,12 @@ class ShuffleReadMetrics extends Serializable { */ var remoteFetchTime: Long = _ - /** Total number of remote bytes read from the shuffle by this task */ + /** + * Total number of remote bytes read from the shuffle by this task + */ var remoteBytesRead: Long = _ - def toJson: JValue = { + override def toJson = { ("Shuffle Finish Time" -> shuffleFinishTime) ~ ("Total Blocks Fetched" -> totalBlocksFetched) ~ ("Remote Blocks Fetched" -> remoteBlocksFetched) ~ @@ -116,14 +143,18 @@ class ShuffleReadMetrics extends Serializable { } } -class ShuffleWriteMetrics extends Serializable { - /** Number of bytes written for the shuffle by this task */ +class ShuffleWriteMetrics extends Serializable with JsonSerializable { + /** + * Number of bytes written for the shuffle by this task + */ var shuffleBytesWritten: Long = _ - /** Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ + /** + * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds + */ var shuffleWriteTime: Long = _ - def toJson: JValue = { + override def toJson = { ("Shuffle Bytes Written" -> shuffleBytesWritten) ~ ("Shuffle Write Time" -> shuffleWriteTime) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 9f4643738f970..ee7f4ed3e947b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,23 +17,20 @@ package org.apache.spark.scheduler -import net.liftweb.json.JsonAST._ import net.liftweb.json.JsonDSL._ /** * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult { - def toJson: JValue -} +private[spark] sealed trait JobResult extends JsonSerializable private[spark] case object JobSucceeded extends JobResult { - def toJson = ("Status" -> "Success") + override def toJson = ("Status" -> "Success") } private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult { - def toJson = { + override def toJson = { ("Status" -> "Failed") ~ ("Exception" -> exception.getMessage) ~ ("Failed Stage ID" -> failedStageId) 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 81409431f4375..d2cf257652be4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -25,31 +25,33 @@ import org.apache.spark.executor.TaskMetrics import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ +import org.apache.spark.storage.StorageStatus -sealed trait SparkListenerEvent { - def toJson: JValue -} +trait JsonSerializable { def toJson: JValue } + +sealed trait SparkListenerEvent extends JsonSerializable case class SparkListenerStageSubmitted( stageInfo: StageInfo, properties: Properties) extends SparkListenerEvent { - def toJson = { + override def toJson = { + val propertiesJson = Utils.propertiesToJson(properties) ("Event" -> "Stage Submitted") ~ ("Stage Info" -> stageInfo.toJson) ~ - ("Properties" -> Utils.propertiesToJson(properties)) + ("Properties" -> propertiesJson) } } case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent { - def toJson = { + override def toJson = { ("Event" -> "Stage Completed") ~ ("Stage Info" -> stageInfo.toJson) } } case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent { - def toJson = { + override def toJson = { ("Event" -> "Task Start") ~ ("Stage ID" -> stageId) ~ ("Task Info" -> taskInfo.toJson) @@ -57,7 +59,7 @@ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends Spar } case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent { - def toJson = { + override def toJson = { ("Event" -> "Task Getting Result") ~ ("Task Info" -> taskInfo.toJson) } @@ -70,11 +72,12 @@ case class SparkListenerTaskEnd( taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvent { - def toJson = { + override def toJson = { + val _reason = Utils.getFormattedClassName(reason) ("Event" -> "Task End") ~ ("Stage ID" -> stageId) ~ ("Task Type" -> taskType) ~ - ("Task End Reason" -> reason.toString) ~ + ("Task End Reason" -> _reason) ~ ("Task Info" -> taskInfo.toJson) ~ ("Task Metrics" -> taskMetrics.toJson) } @@ -85,16 +88,18 @@ case class SparkListenerJobStart( stageIds: Seq[Int], properties: Properties) extends SparkListenerEvent { - def toJson = { + override def toJson = { + val stageIdsJson = JArray(stageIds.map(JInt(_)).toList) + val propertiesJson = Utils.propertiesToJson(properties) ("Event" -> "Job Start") ~ ("Job ID" -> jobId) ~ - ("Stage IDs" -> JArray(stageIds.map(JInt(_)).toList)) ~ - ("Properties" -> Utils.propertiesToJson(properties)) + ("Stage IDs" -> stageIdsJson) ~ + ("Properties" -> propertiesJson) } } case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent { - def toJson = { + override def toJson = { ("Event" -> "Job End") ~ ("Job ID" -> jobId) ~ ("Job Result" -> jobResult.toJson) @@ -103,20 +108,36 @@ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkLi /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent { - def toJson = ("Event" -> "Shutdown") + override def toJson = ("Event" -> "Shutdown") +} + +/** An event used in the ExecutorUI to fetch storage status from SparkEnv */ +private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq[StorageStatus]) + extends SparkListenerEvent { + override def toJson = { + val storageStatusListJson = JArray(storageStatusList.map(_.toJson).toList) + ("Event" -> "Storage Status Fetch") ~ + ("Storage Status List" -> storageStatusListJson) + } } /** * Interface for listening to events from the Spark scheduler. */ trait SparkListener { - /** Called when a stage is completed, with information on the completed stage */ + /** + * Called when a stage is completed, with information on the completed stage + */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } - /** Called when a stage is submitted */ + /** + * Called when a stage is submitted + */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - /** Called when a task starts */ + /** + * Called when a task starts + */ def onTaskStart(taskStart: SparkListenerTaskStart) { } /** @@ -125,13 +146,19 @@ trait SparkListener { */ def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } - /** Called when a task ends */ + /** + * Called when a task ends + */ def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } - /** Called when a job starts */ + /** + * Called when a job starts + */ def onJobStart(jobStart: SparkListenerJobStart) { } - /** Called when a job ends */ + /** + * Called when a job ends + */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } } @@ -243,7 +270,9 @@ private[spark] object StatsReportListener extends Logging { val minutes = seconds * 60 val hours = minutes * 60 - /** Reformat a time interval in milliseconds to a prettier format for output */ + /** + * Reformat a time interval in milliseconds to a prettier format for output + */ def millisToString(ms: Long) = { val (size, units) = if (ms > hours) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 325d9b17c0fbc..ba63bd0ca561c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -27,12 +27,19 @@ import net.liftweb.json.JsonDSL._ /** * Stores information about a stage to pass from the scheduler to SparkListeners. */ -class StageInfo(stage: Stage) { +class StageInfo(stage: Stage) extends JsonSerializable { val stageId = stage.id - /** Store the metrics for all tasks that have completed, including redundant, speculated tasks. */ + + /** + * Store the metrics for all tasks that have completed, including redundant, speculated tasks. + */ val taskInfos = mutable.Buffer[(TaskInfo, TaskMetrics)]() - /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ + + /** + * When this stage was submitted from the DAGScheduler to a TaskScheduler. + */ var submissionTime: Option[Long] = None + var completionTime: Option[Long] = None val rddName = stage.rdd.name val name = stage.name @@ -40,7 +47,7 @@ class StageInfo(stage: Stage) { val numTasks = stage.numTasks var emittedTaskSizeWarning = false - def toJson: JValue = { + override def toJson = { val (taskInfoList, taskMetricsList) = taskInfos.toList.unzip val taskInfoJson = JArray(taskInfoList.map(_.toJson)) val taskMetricsJson = JArray(taskMetricsList.map(_.toJson)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index aa4e73ff0c804..6a3c9e858f0ef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ /** * Information about a running task attempt inside a TaskSet. @@ -30,7 +29,8 @@ class TaskInfo( val launchTime: Long, val executorId: String, val host: String, - val taskLocality: TaskLocality.TaskLocality) { + val taskLocality: TaskLocality.TaskLocality) + extends JsonSerializable { /** * The time when the task started remotely getting the result. Will not be set if the @@ -94,7 +94,8 @@ class TaskInfo( def timeRunning(currentTime: Long): Long = currentTime - launchTime - def toJson: JValue = { + override def toJson = { + val _duration = if (finished) duration else 0L ("Task ID" -> taskId) ~ ("Index" -> index) ~ ("Launch Time" -> launchTime) ~ @@ -103,6 +104,7 @@ class TaskInfo( ("Locality" -> taskLocality.toString) ~ ("Status" -> status) ~ ("Getting Result Time" -> gettingResultTime) ~ + ("Duration" -> _duration) ~ ("Finish Time" -> finishTime) ~ ("Failed" -> failed) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 301d784b350a3..b289a19b3594b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -18,6 +18,9 @@ package org.apache.spark.storage import java.util.UUID +import org.apache.spark.scheduler.JsonSerializable + +import net.liftweb.json.JsonDSL._ /** * Identifies a particular Block of data, usually associated with a single file. @@ -26,7 +29,7 @@ import java.util.UUID * * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ -private[spark] sealed abstract class BlockId { +private[spark] sealed abstract class BlockId extends JsonSerializable { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -42,6 +45,8 @@ private[spark] sealed abstract class BlockId { case o: BlockId => getClass == o.getClass && name.equals(o.name) case _ => false } + + override def toJson = ("Name" -> name) } private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 74207f59af170..afee46d74032d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,6 +20,9 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap import org.apache.spark.util.Utils +import org.apache.spark.scheduler.JsonSerializable + +import net.liftweb.json.JsonDSL._ /** * This class represent an unique identifier for a BlockManager. @@ -34,7 +37,7 @@ private[spark] class BlockManagerId private ( private var host_ : String, private var port_ : Int, private var nettyPort_ : Int - ) extends Externalizable { + ) extends Externalizable with JsonSerializable { private def this() = this(null, null, 0, 0) // For deserialization only @@ -49,7 +52,6 @@ private[spark] class BlockManagerId private ( // DEBUG code Utils.checkHost(host) assert (port > 0) - host + ":" + port } @@ -86,13 +88,20 @@ private[spark] class BlockManagerId private ( case _ => false } + + override def toJson = { + ("Executor ID" -> executorId) ~ + ("Host Port" -> hostPort) ~ + ("Host" -> host) ~ + ("Port" -> port) + } } private[spark] object BlockManagerId { /** - * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuraiton. + * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuration. * * @param execId ID of the executor. * @param host Host name of the block manager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 893418fb8cad9..0b600df6de143 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -30,6 +30,9 @@ import akka.pattern.ask import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.scheduler.JsonSerializable + +import net.liftweb.json.JsonDSL._ /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -306,11 +309,17 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } } - private[spark] object BlockManagerMasterActor { case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) + extends JsonSerializable { + override def toJson = { + ("Storage Level" -> storageLevel.toJson) ~ + ("Memory Size" -> memSize) ~ + ("Disk Size" -> diskSize) + } + } class BlockManagerInfo( val blockManagerId: BlockManagerId, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1b7934d59fa1d..4b2a80b226a27 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,6 +19,9 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import net.liftweb.json.JsonDSL._ +import org.apache.spark.scheduler.JsonSerializable + /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory @@ -32,7 +35,7 @@ class StorageLevel private( private var useMemory_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) - extends Externalizable { + extends Externalizable with JsonSerializable { // TODO: Also add fields for caching priority, dataset ID, and flushing. private def this(flags: Int, replication: Int) { @@ -105,6 +108,13 @@ class StorageLevel private( result += "%sx Replicated".format(replication) result } + + override def toJson = { + ("Use Disk" -> useDisk) ~ + ("Use Memory" -> useMemory) ~ + ("Deserialized" -> deserialized) ~ + ("Replication" -> replication) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 50a0cdb3095cd..5dc5522fa3d9e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,13 +17,17 @@ package org.apache.spark.storage -import org.apache.spark.{SparkContext} +import org.apache.spark.SparkContext import BlockManagerMasterActor.BlockStatus import org.apache.spark.util.Utils +import org.apache.spark.scheduler.JsonSerializable + +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, - blocks: Map[BlockId, BlockStatus]) { + blocks: Map[BlockId, BlockStatus]) extends JsonSerializable { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) @@ -41,6 +45,20 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, case (rdd: RDDBlockId, status) => Some(rdd, status) case _ => None } + + override def toJson = { + val blocksJson = JArray( + blocks.toList.map { case (id, status) => + ("Block ID" -> id.toJson) ~ + ("Status" -> status.toJson) + }) + ("Block Manager ID" -> blockManagerId.toJson) ~ + ("Maximum Memory" -> maxMem) ~ + ("Memory Used" -> memUsed) ~ + ("Memory Remaining" -> memRemaining) ~ + ("Disk Used" -> diskUsed) ~ + ("Blocks" -> blocksJson) + } } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 9ccf536a2a6ca..92394dea76f28 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -36,11 +36,12 @@ import net.liftweb.json.JsonDSL._ private[spark] class ExecutorsUI(val sc: SparkContext) { + private val data = new ExecutorsData private var _listener: Option[ExecutorsListener] = None def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener(sc)) + _listener = Some(new ExecutorsListener(this)) sc.addSparkListener(listener) } @@ -48,19 +49,23 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { ("/executors", (request: HttpServletRequest) => render(request)) ) - /** Render an HTML page that encodes executor information */ + /** + * Render an HTML page that encodes executor information + */ def render(request: HttpServletRequest): Seq[Node] = { - listener.updateStorageStatusFromEnv() - val summaryJson = listener.summaryJson - val executorsJson = listener.executorIdToJson.values.toSeq + listener.getExecutorStorageStatus() + val summaryJson = data.summaryJson + val executorsJson = data.executorIdToJson.values.toSeq renderFromJson(summaryJson, executorsJson) } - /** Render an HTML page that encodes executor information from the given JSON representations */ + /** + * Render an HTML page that encodes executor information from the given JSON representations + */ def renderFromJson(summaryJson: JValue, executorsJson: Seq[JValue]): Seq[Node] = { - val memoryAvailable = Utils.extractLongFromJson(summaryJson, "Memory Available").getOrElse(0L) - val memoryUsed = Utils.extractLongFromJson(summaryJson, "Memory Used").getOrElse(0L) - val diskSpaceUsed = Utils.extractLongFromJson(summaryJson, "Disk Space Used").getOrElse(0L) + val maximumMemory = Utils.extractLongFromJson(summaryJson \ "Maximum Memory") + val memoryUsed = Utils.extractLongFromJson(summaryJson \ "Memory Used") + val diskSpaceUsed = Utils.extractLongFromJson(summaryJson \ "Disk Space Used") val execTable = UIUtils.listingTable[JValue](execHeader, execRow, executorsJson) val content =
@@ -68,8 +73,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
  • Memory: - {Utils.bytesToString(memoryAvailable)} Used - ({Utils.bytesToString(memoryUsed)} Total) + {Utils.bytesToString(memoryUsed)} Used + ({Utils.bytesToString(maximumMemory)} Total)
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
@@ -99,13 +104,16 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { "Shuffle Read", "Shuffle Write") - /** Render an HTML table row representing an executor from the given JSON representation */ + /** + * Render an HTML table row representing an executor from the given JSON representation + */ private def execRow(executorJson: JValue): Seq[Node] = { - def getString(field: String) = Utils.extractStringFromJson(executorJson, field).getOrElse("") - def getLong(field: String) = Utils.extractLongFromJson(executorJson, field).getOrElse(0L) - def getInt(field: String) = Utils.extractIntFromJson(executorJson, field).getOrElse(0) + def getString(field: String) = Utils.extractStringFromJson(executorJson \ field) + def getLong(field: String) = Utils.extractLongFromJson(executorJson \ field) + def getInt(field: String) = Utils.extractIntFromJson(executorJson \ field) + + val maximumMemory = getLong("Maximum Memory") val memoryUsed = getLong("Memory Used") - val memoryAvailable = getLong("Memory Available") val diskUsed = getLong("Disk Used") val activeTasks = getInt("Active Tasks") val failedTasks = getInt("Failed Tasks") @@ -118,7 +126,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {getString("RDD Blocks")} {Utils.bytesToString(memoryUsed)} / - {Utils.bytesToString(memoryAvailable)} + {Utils.bytesToString(maximumMemory)} {Utils.bytesToString(diskUsed)} @@ -134,23 +142,24 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } /** - * A SparkListener that maintains and logs information to be displayed on the Executors UI. + * A vessel for storing executors information in JSON format. * - * Both intermediate data that resides in memory and persisted data that resides on disk are - * in JSON format. + * The source of data can come from two sources: (1) If the job is live, ExecutorsListener + * invokes ExecutorsData to update its data on certain Spark events. (2) If we are rendering + * the UI for a job from the past, ExecutorsData parses from these log files the necessary + * states to reconstruct the UI. */ - private[spark] class ExecutorsListener(sc: SparkContext) extends SparkListener with Logging { + private[spark] class ExecutorsData extends Logging { var summaryJson: JValue = JNothing val executorIdToJson = HashMap[String, JValue]() - private val logger = new FileLogger("executors-ui") - /** Return the JSON representation of a newly discovered executor */ + /** JSON representation of a newly discovered executor */ private def newExecutorJson(execId: String): JValue = { ("Executor ID" -> execId) ~ ("Address" -> "") ~ ("RDD Blocks" -> "") ~ + ("Maximum Memory" -> 0L) ~ ("Memory Used" -> 0L) ~ - ("Memory Available" -> 0L) ~ ("Disk Used" -> 0L) ~ ("Active Tasks" -> 0) ~ ("Failed Tasks" -> 0) ~ @@ -161,64 +170,58 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } /** - * Update the summary and per-executor storage status from SparkEnv. This involves querying - * the driver and waiting for a reply, and so should be called sparingly. + * Update the summary and per-executor storage status from JSON */ - def updateStorageStatusFromEnv() { + def storageStatusFetch(storageStatusFetchJson: JValue) { + val storageStatusList = + Utils.extractListFromJson(storageStatusFetchJson \ "Storage Status List") // Update summary storage information - val storageStatusList = sc.getExecutorStorageStatus - val memoryAvailable = storageStatusList.map(_.maxMem).fold(0L)(_+_) - val memoryUsed = storageStatusList.map(_.memUsed).fold(0L)(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_+_) + val maximumMemory = storageStatusList.map { status => + Utils.extractLongFromJson(status \ "Maximum Memory") + }.fold(0L)(_+_) + val memoryUsed = storageStatusList.map { status => + Utils.extractLongFromJson(status \ "Memory Used") + }.fold(0L)(_+_) + val diskSpaceUsed = storageStatusList.flatMap { status => + Utils.extractListFromJson(status \ "Blocks").map { block => + Utils.extractLongFromJson(block \ "Status" \ "Disk Size") + } + }.fold(0L)(_+_) summaryJson = - ("Memory Available" -> memoryAvailable) ~ + ("Maximum Memory" -> maximumMemory) ~ ("Memory Used" -> memoryUsed) ~ ("Disk Space Used" -> diskSpaceUsed) // Update storage status for each executor storageStatusList.foreach { status => - val execId = status.blockManagerId.executorId - val address = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size - val memoryUsed = status.memUsed - val memoryAvailable = status.maxMem - val diskUsed = status.diskUsed - val json = executorIdToJson.getOrElse(execId, newExecutorJson(execId)) - executorIdToJson(execId) = json.transform { - case JField("Address", _) => JField("Address", JString(address)) - case JField("RDD Blocks", _) => JField("RDD Blocks", JInt(rddBlocks)) - case JField("Memory Used", _) => JField("Memory Used", JInt(memoryUsed)) - case JField("Memory Available", _) => JField("Memory Available", JInt(memoryAvailable)) - case JField("Disk Used", _) => JField("Disk Used", JInt(diskUsed)) + val execId = formatExecutorId( + Utils.extractStringFromJson(status \ "Block Manager ID" \ "Executor ID")) + if (execId != "") { + val address = Utils.extractStringFromJson(status \ "Block Manager ID" \ "Host Port") + val rddBlocks = Utils.extractListFromJson(status \ "Blocks").size + val maximumMemory = Utils.extractLongFromJson(status \ "Maximum Memory") + val memoryUsed = Utils.extractLongFromJson(status \ "Memory Used") + val diskUsed = Utils.extractLongFromJson(status \ "Disk Used") + val json = executorIdToJson.getOrElse(execId, newExecutorJson(execId)) + executorIdToJson(execId) = json.transform { + case JField("Address", _) => JField("Address", JString(address)) + case JField("RDD Blocks", _) => JField("RDD Blocks", JInt(rddBlocks)) + case JField("Maximum Memory", _) => JField("Maximum Memory", JInt(maximumMemory)) + case JField("Memory Used", _) => JField("Memory Used", JInt(memoryUsed)) + case JField("Disk Used", _) => JField("Disk Used", JInt(diskUsed)) + } } - logJson(executorIdToJson(execId)) } } - override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() - - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() - - override def onStageSubmitted(stageStart: SparkListenerStageSubmitted) = { - updateStorageStatusFromEnv() - logger.flush() - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { - updateStorageStatusFromEnv() - logger.flush() - } - - override def onTaskStart(taskStart: SparkListenerTaskStart) = { - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). This results in - * duplicate rows for the same executor. Thus, in this mode, we aggregate these two - * rows and use the executor ID of "" to be consistent. - */ - val execId = if (sc.isLocal) "" else taskStart.taskInfo.executorId + /** + * Update executor information in response to a task start event + */ + def taskStart(taskStartJson: JValue) { + val execId = formatExecutorId( + Utils.extractStringFromJson(taskStartJson \ "Task Info" \ "Executor ID")) val json = executorIdToJson.getOrElse(execId, { // The executor ID according to the task is different from that according to SparkEnv // This should never happen under normal circumstances... @@ -228,22 +231,22 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { executorIdToJson(execId) = json.transform { case JField("Active Tasks", JInt(s)) => JField("Active Tasks", JInt(s + 1)) } - logJson(executorIdToJson(execId)) } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = { - val execId = if (sc.isLocal) "" else taskEnd.taskInfo.executorId - val newDuration = taskEnd.taskInfo.duration - var newShuffleRead = 0L - var newShuffleWrite = 0L - if (taskEnd.taskMetrics != null) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach(newShuffleRead += _.remoteBytesRead) - taskEnd.taskMetrics.shuffleWriteMetrics.foreach(newShuffleWrite += _.shuffleBytesWritten) - } - val success = taskEnd.reason match { - case _: ExceptionFailure => false - case _ => true - } + /** + * Update executor information in response to a task end event + */ + def taskEnd(taskEndJson: JValue) { + val execId = formatExecutorId( + Utils.extractStringFromJson(taskEndJson \ "Task Info" \ "Executor ID")) + val duration = Utils.extractLongFromJson(taskEndJson \ "Task Info" \ "Duration") + val taskEndReason = Utils.extractStringFromJson(taskEndJson \ "Task End Reason") + val failed = taskEndReason == ExceptionFailure.getClass.getSimpleName + val shuffleRead = Utils.extractLongFromJson( + taskEndJson \ "Task Metrics" \ "Shuffle Read Metrics" \ "Remote Bytes Read") + val shuffleWrite = Utils.extractLongFromJson( + taskEndJson \ "Task Metrics" \ "Shuffle Write Metrics" \ "Shuffle Bytes Written") + val json = executorIdToJson.getOrElse(execId, { // Information for this executor has vanished over the course of the task execution // This should never happen under normal circumstances... @@ -252,24 +255,82 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { }) executorIdToJson(execId) = json.transform { case JField("Active Tasks", JInt(t)) if t > 0 => JField("Active Tasks", JInt(t - 1)) - case JField("Failed Tasks", JInt(t)) if !success => JField("Failed Tasks", JInt(t + 1)) - case JField("Complete Tasks", JInt(t)) if success => JField("Complete Tasks", JInt(t + 1)) - case JField("Task Time", JInt(s)) => JField("Task Time", JInt(s + newDuration)) - case JField("Shuffle Read", JInt(s)) => JField("Shuffle Read", JInt(s + newShuffleRead)) - case JField("Shuffle Write", JInt(s)) => JField("Shuffle Write", JInt(s + newShuffleWrite)) + case JField("Failed Tasks", JInt(t)) if failed => JField("Failed Tasks", JInt(t + 1)) + case JField("Complete Tasks", JInt(t)) if !failed => JField("Complete Tasks", JInt(t + 1)) + case JField("Task Time", JInt(s)) => JField("Task Time", JInt(s + duration)) + case JField("Shuffle Read", JInt(s)) => JField("Shuffle Read", JInt(s + shuffleRead)) + case JField("Shuffle Write", JInt(s)) => JField("Shuffle Write", JInt(s + shuffleWrite)) } - logJson(executorIdToJson(execId)) } - /** Log summary storage status **/ - def logSummary() = logJson(summaryJson) + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). This results in + * duplicate rows for the same executor. Thus, in this mode, we aggregate these two + * rows and use the executor ID of "" to be consistent. + */ + private def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId + } + } + + /** + * A SparkListener that logs information to be displayed on the Executors UI. + * + * Currently, ExecutorsListener only fetches executor storage information from the driver + * on stage submit and completion. However, this is arbitrary and needs not be true. More + * specifically, each stage could be very long, in which case it would take a while before + * this information is updated and persisted to disk. An alternative approach would be to + * fetch every constant number of task events. + */ + private[spark] class ExecutorsListener(ui: ExecutorsUI) extends SparkListener with Logging { + private val logger = new FileLogger("executors-ui") + + /** + * Invoke SparkEnv to ask the driver for executor storage status. This should be + * called sparingly. + */ + def getExecutorStorageStatus() = { + val storageStatusList = ui.sc.getExecutorStorageStatus + val event = SparkListenerStorageStatusFetch(storageStatusList) + onStorageStatusFetch(event) + } + + override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() - /** Log storage status for the executor with the given ID */ - def logExecutor(execId: String) = logJson(executorIdToJson.getOrElse(execId, JNothing)) + override def onStageSubmitted(stageStart: SparkListenerStageSubmitted) = { + getExecutorStorageStatus() + logger.flush() + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { + getExecutorStorageStatus() + logger.flush() + } + + override def onTaskStart(taskStart: SparkListenerTaskStart) = { + val eventJson = taskStart.toJson + ui.data.taskStart(eventJson) + logEvent(eventJson) + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = { + val eventJson = taskEnd.toJson + ui.data.taskEnd(eventJson) + logEvent(eventJson) + } + + def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) = { + val eventJson = storageStatusFetch.toJson + ui.data.storageStatusFetch(eventJson) + logEvent(eventJson) + } - private def logJson(json: JValue) = { - if (json != JNothing) { - logger.logLine(compactRender(json)) + private def logEvent(event: JValue) = { + if (event != JNothing) { + logger.logLine(compactRender(event)) } } } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index c1d179ba1d687..d8718efa44082 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -25,7 +25,7 @@ import java.util.Date * A generic class for logging information to file * @param user User identifier if SPARK_LOG_DIR is not set, in which case log directory * defaults to /tmp/spark-[user] - * @param name Name of logger, also the name of the log file + * @param name Name of logger, also the base name of the log files * @param flushFrequency How many writes until the results are flushed to disk */ class FileLogger(user: String, name: String, flushFrequency: Int = 100) { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8086b8c7a341f..959d6def45817 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -873,6 +873,11 @@ private[spark] object Utils extends Logging { count } + /** Return the class name of the given object, removing all dollar signs */ + def getFormattedClassName(obj: AnyRef) = { + obj.getClass.getSimpleName.replace("$", "") + } + /** Convert a (String, String) map to a JSON object */ def mapToJson(m: Map[String, String]): JValue = { val jsonFields = m.map { case (k, v) => @@ -888,36 +893,28 @@ private[spark] object Utils extends Logging { }.getOrElse(JNothing) } - /** Extract a field from the given JSON AST as a scala option */ - def extractFromJson(json: JValue, field: String): Option[JValue] = { - (json \ field) match { - case JNothing => None - case value => Some(value) - } + /** Extract a list from the given JSON AST */ + def extractListFromJson(json: JValue, default: List[JValue] = List()): List[JValue] = { + if (json == JNothing) default else json.extract[List[JValue]] } - /** Extracts a string from the given JSON AST */ - def extractStringFromJson(json: JValue, field: String): Option[String] = { - extractFromJson(json, field).map(_.extract[String]) + /** Extract a string from the given JSON AST */ + def extractStringFromJson(json: JValue, default: String = ""): String = { + if (json == JNothing) default else json.extract[String] } - /** Extracts a double from the given JSON AST */ - def extractDoubleFromJson(json: JValue, field: String): Option[Double] = { - extractFromJson(json, field).map(_.extract[Double]) + /** Extract a double from the given JSON AST */ + def extractDoubleFromJson(json: JValue, default: Double = 0.0): Double = { + if (json == JNothing) default else json.extract[Double] } /** Extracts a long from the given JSON AST */ - def extractLongFromJson(json: JValue, field: String): Option[Long] = { - extractFromJson(json, field).map(_.extract[Long]) - } - - /** Extracts an int from the given JSON AST */ - def extractIntFromJson(json: JValue, field: String): Option[Int] = { - extractFromJson(json, field).map(_.extract[Int]) + def extractLongFromJson(json: JValue, default: Long = 0L): Long = { + if (json == JNothing) default else json.extract[Long] } - /** Extracts a boolean from the given JSON AST */ - def extractBooleanFromJson(json: JValue, field: String): Option[Boolean] = { - extractFromJson(json, field).map(_.extract[Boolean]) + /** Extract an int from the given JSON AST */ + def extractIntFromJson(json: JValue, default: Int = 0): Int = { + if (json == JNothing) default else json.extract[Int] } } From de8a1cdb833d80423aba629ba932b6f403ecd4ab Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Feb 2014 19:22:50 -0800 Subject: [PATCH 12/68] Serialize events both to and from JSON (rather than just to) This requires every field of every event to be completely reconstructible from its JSON representation. This commit may contain incomplete state. --- .../org/apache/spark/TaskEndReason.scala | 100 ++++- .../org/apache/spark/executor/Executor.scala | 8 +- .../apache/spark/executor/TaskMetrics.scala | 61 ++- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../apache/spark/scheduler/JobResult.scala | 35 +- .../spark/scheduler/SparkListener.scala | 127 +++++- .../apache/spark/scheduler/StageInfo.scala | 92 +++-- .../org/apache/spark/scheduler/TaskInfo.scala | 28 +- .../apache/spark/scheduler/TaskLocality.scala | 19 +- .../apache/spark/storage/BlockManagerId.scala | 19 +- .../scala/org/apache/spark/ui/SparkUI.scala | 15 +- .../apache/spark/ui/env/EnvironmentUI.scala | 139 ++++--- .../apache/spark/ui/exec/ExecutorsUI.scala | 367 +++++++----------- .../spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 67 ++-- .../spark/scheduler/JobLoggerSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 12 +- 18 files changed, 713 insertions(+), 386 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 3fd6f5eb472f4..0dbecb3ac3f15 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -19,50 +19,124 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId +import org.apache.spark.scheduler.JsonSerializable +import org.apache.spark.util.Utils + +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats /** * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason +sealed trait TaskEndReason extends JsonSerializable { + override def toJson = "Reason" -> Utils.getFormattedClassName(this) +} + +case object TaskEndReason { + def fromJson(json: JValue): TaskEndReason = { + implicit val format = DefaultFormats + val success = Utils.getFormattedClassName(Success) + val resubmitted = Utils.getFormattedClassName(Resubmitted) + val fetchFailed = Utils.getFormattedClassName(FetchFailed) + val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) + val taskResultLost = Utils.getFormattedClassName(TaskResultLost) + val taskKilled = Utils.getFormattedClassName(TaskKilled) + val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) + val unknownReason = Utils.getFormattedClassName(UnknownReason) + + (json \ "Reason").extract[String] match { + case `success` => Success + case `resubmitted` => Resubmitted + case `fetchFailed` => fetchFailedFromJson(json) + case `exceptionFailure` => exceptionFailureFromJson(json) + case `taskResultLost` => TaskResultLost + case `taskKilled` => TaskKilled + case `executorLostFailure` => ExecutorLostFailure + case `unknownReason` => UnknownReason + } + } -private[spark] case object Success extends TaskEndReason + private def fetchFailedFromJson(json: JValue): TaskEndReason = { + implicit val format = DefaultFormats + new FetchFailed( + BlockManagerId.fromJson(json \ "Block Manager Address"), + (json \ "Shuffle ID").extract[Int], + (json \ "Map ID").extract[Int], + (json \ "Reduce ID").extract[Int]) + } -private[spark] -case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it + private def exceptionFailureFromJson(json: JValue): TaskEndReason = { + implicit val format = DefaultFormats + val metrics = (json \ "Metrics") match { + case JNothing => None + case value: JValue => Some(TaskMetrics.fromJson(value)) + } + val stackTrace = Utils.stackTraceFromJson(json \ "Stack Trace") + new ExceptionFailure( + (json \ "Class Name").extract[String], + (json \ "Description").extract[String], + stackTrace, + metrics + ) + } +} -private[spark] case class FetchFailed( +case object Success extends TaskEndReason + +// Task was finished earlier but we've now lost it +case object Resubmitted extends TaskEndReason + +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) - extends TaskEndReason + extends TaskEndReason { + override def toJson = { + super.toJson ~ + ("Block Manager Address" -> bmAddress.toJson) ~ + ("Shuffle ID" -> shuffleId) ~ + ("Map ID" -> mapId) ~ + ("Reduce ID" -> reduceId) + } +} -private[spark] case class ExceptionFailure( +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], metrics: Option[TaskMetrics]) - extends TaskEndReason + extends TaskEndReason { + override def toJson = { + val stackTraceJson = Utils.stackTraceToJson(stackTrace) + val metricsJson = metrics.map(_.toJson).getOrElse(JNothing) + super.toJson ~ + ("Class Name" -> className) ~ + ("Description" -> description) ~ + ("Stack Trace" -> stackTraceJson) ~ + ("Metrics" -> metricsJson) + } +} /** * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -private[spark] case object TaskResultLost extends TaskEndReason +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +case object TaskKilled extends TaskEndReason /** * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +case object ExecutorLostFailure extends TaskEndReason /** * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -private[spark] case object UnknownReason extends TaskEndReason - +case object UnknownReason extends TaskEndReason 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 989d666f15600..801f887178b51 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -225,10 +225,10 @@ private[spark] class Executor( for (m <- task.metrics) { m.hostname = Utils.localHostName() - m.executorDeserializeTime = (taskStart - startTime).toInt - m.executorRunTime = (taskFinish - taskStart).toInt + m.executorDeserializeTime = taskStart - startTime + m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime - m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt + m.resultSerializationTime = afterSerialization - beforeSerialization } val accumUpdates = Accumulators.values @@ -264,7 +264,7 @@ private[spark] class Executor( } case t: Throwable => { - val serviceTime = (System.currentTimeMillis() - taskStart).toInt + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { m.executorRunTime = serviceTime diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 282b7e5dbee07..62b1bb230ef86 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -21,6 +21,7 @@ import org.apache.spark.scheduler.JsonSerializable import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats class TaskMetrics extends Serializable with JsonSerializable { /** @@ -31,12 +32,12 @@ class TaskMetrics extends Serializable with JsonSerializable { /** * Time taken on the executor to deserialize this task */ - var executorDeserializeTime: Int = _ + var executorDeserializeTime: Long = _ /** * Time the executor spends actually running the task (including fetching shuffle data) */ - var executorRunTime: Int = _ + var executorRunTime: Long = _ /** * The number of bytes this task transmitted back to the driver as the TaskResult @@ -88,10 +89,6 @@ class TaskMetrics extends Serializable with JsonSerializable { } } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics -} - class ShuffleReadMetrics extends Serializable with JsonSerializable { /** * Absolute time when this task finished reading shuffle data @@ -159,3 +156,55 @@ class ShuffleWriteMetrics extends Serializable with JsonSerializable { ("Shuffle Write Time" -> shuffleWriteTime) } } + +object TaskMetrics { + private[spark] def empty(): TaskMetrics = new TaskMetrics + + def fromJson(json: JValue): TaskMetrics = { + implicit val format = DefaultFormats + val metrics = new TaskMetrics + metrics.hostname = (json \ "Host Name").extract[String] + metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] + metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] + metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] + metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] + metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] + metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] + metrics.shuffleReadMetrics = + json \ "Shuffle Read Metrics" match { + case JNothing => None + case value: JValue => Some(ShuffleReadMetrics.fromJson(value)) + } + metrics.shuffleWriteMetrics = + json \ "Shuffle Write Metrics" match { + case JNothing => None + case value: JValue => Some(ShuffleWriteMetrics.fromJson(value)) + } + metrics + } +} + +object ShuffleReadMetrics { + def fromJson(json: JValue): ShuffleReadMetrics = { + implicit val format = DefaultFormats + val metrics = new ShuffleReadMetrics + metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] + metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] + metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] + metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] + metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] + metrics.remoteFetchTime = (json \ "Remote Fetch Time").extract[Long] + metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] + metrics + } +} + +object ShuffleWriteMetrics { + def fromJson(json: JValue): ShuffleWriteMetrics = { + implicit val format = DefaultFormats + val metrics = new ShuffleWriteMetrics + metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] + metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] + metrics + } +} 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 86e86bbaa16db..2acfd38ac3273 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -250,7 +250,7 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = new StageInfo(stage) + stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -826,7 +826,7 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics + stageToInfos(stage).taskInfo += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index ee7f4ed3e947b..ab37336ebaeb8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -18,21 +18,46 @@ package org.apache.spark.scheduler import net.liftweb.json.JsonDSL._ +import org.apache.spark.util.Utils +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult extends JsonSerializable +private[spark] sealed trait JobResult extends JsonSerializable { + override def toJson = "Result" -> Utils.getFormattedClassName(this) +} + +private[spark] object JobResult { + def fromJson(json: JValue): JobResult = { + implicit val format = DefaultFormats + val jobSucceededString = Utils.getFormattedClassName(JobSucceeded) + val jobFailedString = Utils.getFormattedClassName(JobFailed) + + (json \ "Result").extract[String] match { + case `jobSucceededString` => JobSucceeded + case `jobFailedString` => jobFailedFromJson(json) + } + } -private[spark] case object JobSucceeded extends JobResult { - override def toJson = ("Status" -> "Success") + private def jobFailedFromJson(json: JValue): JobResult = { + implicit val format = DefaultFormats + new JobFailed( + Utils.exceptionFromJson(json \ "Exception"), + (json \ "Failed Stage ID").extract[Int]) + } } +private[spark] case object JobSucceeded extends JobResult + private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult { override def toJson = { - ("Status" -> "Failed") ~ - ("Exception" -> exception.getMessage) ~ + val exceptionJson = Utils.exceptionToJson(exception) + + super.toJson ~ + ("Exception" -> exceptionJson) ~ ("Failed Stage ID" -> failedStageId) } } 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 d2cf257652be4..d321b7aa21c14 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,10 +26,13 @@ import org.apache.spark.executor.TaskMetrics import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ import org.apache.spark.storage.StorageStatus +import net.liftweb.json.DefaultFormats trait JsonSerializable { def toJson: JValue } -sealed trait SparkListenerEvent extends JsonSerializable +sealed trait SparkListenerEvent extends JsonSerializable { + override def toJson = "Event" -> Utils.getFormattedClassName(this) +} case class SparkListenerStageSubmitted( stageInfo: StageInfo, @@ -37,7 +40,7 @@ case class SparkListenerStageSubmitted( extends SparkListenerEvent { override def toJson = { val propertiesJson = Utils.propertiesToJson(properties) - ("Event" -> "Stage Submitted") ~ + super.toJson ~ ("Stage Info" -> stageInfo.toJson) ~ ("Properties" -> propertiesJson) } @@ -45,14 +48,14 @@ case class SparkListenerStageSubmitted( case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent { override def toJson = { - ("Event" -> "Stage Completed") ~ + super.toJson ~ ("Stage Info" -> stageInfo.toJson) } } case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent { override def toJson = { - ("Event" -> "Task Start") ~ + super.toJson ~ ("Stage ID" -> stageId) ~ ("Task Info" -> taskInfo.toJson) } @@ -60,7 +63,7 @@ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends Spar case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent { override def toJson = { - ("Event" -> "Task Getting Result") ~ + super.toJson ~ ("Task Info" -> taskInfo.toJson) } } @@ -74,7 +77,7 @@ case class SparkListenerTaskEnd( extends SparkListenerEvent { override def toJson = { val _reason = Utils.getFormattedClassName(reason) - ("Event" -> "Task End") ~ + super.toJson ~ ("Stage ID" -> stageId) ~ ("Task Type" -> taskType) ~ ("Task End Reason" -> _reason) ~ @@ -91,7 +94,7 @@ case class SparkListenerJobStart( override def toJson = { val stageIdsJson = JArray(stageIds.map(JInt(_)).toList) val propertiesJson = Utils.propertiesToJson(properties) - ("Event" -> "Job Start") ~ + super.toJson ~ ("Job ID" -> jobId) ~ ("Stage IDs" -> stageIdsJson) ~ ("Properties" -> propertiesJson) @@ -100,15 +103,34 @@ case class SparkListenerJobStart( case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent { override def toJson = { - ("Event" -> "Job End") ~ + super.toJson ~ ("Job ID" -> jobId) ~ ("Job Result" -> jobResult.toJson) } } /** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent { - override def toJson = ("Event" -> "Shutdown") +private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent + +/** An event used in the EnvironmentUI */ +private[spark] case class SparkListenerLoadEnvironment( + jvmInformation: Seq[(String, String)], + sparkProperties: Seq[(String, String)], + systemProperties: Seq[(String, String)], + classpathEntries: Seq[(String, String)]) + extends SparkListenerEvent { + + override def toJson = { + val jvmInformationJson = Utils.mapToJson(jvmInformation.toMap) + val sparkPropertiesJson = Utils.mapToJson(sparkProperties.toMap) + val systemPropertiesJson = Utils.mapToJson(systemProperties.toMap) + val classpathEntriesJson = Utils.mapToJson(classpathEntries.toMap) + super.toJson ~ + ("JVM Information" -> jvmInformationJson) ~ + ("Spark Properties" -> sparkPropertiesJson) ~ + ("System Properties" -> systemPropertiesJson) ~ + ("Classpath Entries" -> classpathEntriesJson) + } } /** An event used in the ExecutorUI to fetch storage status from SparkEnv */ @@ -116,11 +138,90 @@ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq extends SparkListenerEvent { override def toJson = { val storageStatusListJson = JArray(storageStatusList.map(_.toJson).toList) - ("Event" -> "Storage Status Fetch") ~ + super.toJson ~ ("Storage Status List" -> storageStatusListJson) } } +object SparkListenerEvent { + + /** + * Deserialize a SparkListenerEvent from JSON + * TODO: include newly added events! + */ + def fromJson(json: JValue): SparkListenerEvent = { + implicit val format = DefaultFormats + val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) + val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) + val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) + val taskGettingResult = Utils.getFormattedClassName(SparkListenerTaskGettingResult) + val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) + val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) + val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) + val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) + + (json \ "Event").extract[String] match { + case `stageSubmitted` => stageSubmittedFromJson(json) + case `stageCompleted` => stageCompletedFromJson(json) + case `taskStart` => taskStartFromJson(json) + case `taskGettingResult` => taskGettingResultFromJson(json) + case `taskEnd` => taskEndFromJson(json) + case `jobStart` => jobStartFromJson(json) + case `jobEnd` => jobEndFromJson(json) + case `shutdown` => SparkListenerShutdown + } + } + + private def stageSubmittedFromJson(json: JValue) = { + new SparkListenerStageSubmitted( + StageInfo.fromJson(json \ "Stage Info"), + Utils.propertiesFromJson(json \ "Properties")) + } + + private def stageCompletedFromJson(json: JValue) = { + new SparkListenerStageCompleted(StageInfo.fromJson(json \ "Stage Info")) + } + + private def taskStartFromJson(json: JValue) = { + implicit val format = DefaultFormats + new SparkListenerTaskStart( + (json \ "Stage ID").extract[Int], + TaskInfo.fromJson(json \ "Task Info")) + } + + private def taskGettingResultFromJson(json: JValue) = { + new SparkListenerTaskGettingResult(TaskInfo.fromJson(json \ "Task Info")) + } + + private def taskEndFromJson(json: JValue) = { + implicit val format = DefaultFormats + new SparkListenerTaskEnd( + (json \ "Stage ID").extract[Int], + (json \ "Task Type").extract[String], + TaskEndReason.fromJson(json \ "Task End Reason"), + TaskInfo.fromJson(json \ "Task Info"), + TaskMetrics.fromJson(json \ "Task Metrics")) + } + + private def jobStartFromJson(json: JValue) = { + implicit val format = DefaultFormats + val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) + new SparkListenerJobStart( + (json \ "Job ID").extract[Int], + stageIds, + Utils.propertiesFromJson(json \ "Properties") + ) + } + + private def jobEndFromJson(json: JValue) = { + implicit val format = DefaultFormats + new SparkListenerJobEnd( + (json \ "Job ID").extract[Int], + JobResult.fromJson(json \ "Job Result")) + } +} + + /** * Interface for listening to events from the Spark scheduler. */ @@ -184,7 +285,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) // Runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => + val runtimePcts = stageCompleted.stageInfo.taskInfo.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", @@ -206,7 +307,7 @@ private[spark] object StatsReportListener extends Logging { def extractDoubleDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo, TaskMetrics) => Option[Double]) : Option[Distribution] = { - Distribution(stage.stageInfo.taskInfos.flatMap { + Distribution(stage.stageInfo.taskInfo.flatMap { case ((info,metric)) => getMetric(info, metric)}) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index ba63bd0ca561c..c3d6dde8142b5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,48 +17,94 @@ package org.apache.spark.scheduler -import scala.collection._ +import scala.collection.mutable import org.apache.spark.executor.TaskMetrics import net.liftweb.json.JsonAST._ import net.liftweb.json.JsonDSL._ +import org.apache.spark.util.Utils +import net.liftweb.json.DefaultFormats /** - * Stores information about a stage to pass from the scheduler to SparkListeners. + * Stores information about a stage to pass from the scheduler to SparkListeners. Also stores the + * metrics for all tasks that have completed, including redundant, speculated tasks. */ -class StageInfo(stage: Stage) extends JsonSerializable { - val stageId = stage.id - - /** - * Store the metrics for all tasks that have completed, including redundant, speculated tasks. - */ - val taskInfos = mutable.Buffer[(TaskInfo, TaskMetrics)]() +private[spark] +class StageInfo( + val stageId: Int, + val name: String, + val rddName: String, + val numPartitions: Int, + val numTasks: Int, + val taskInfo: mutable.Buffer[(TaskInfo, TaskMetrics)] = + mutable.Buffer[(TaskInfo, TaskMetrics)]() + ) extends JsonSerializable { /** * When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None - var completionTime: Option[Long] = None - val rddName = stage.rdd.name - val name = stage.name - val numPartitions = stage.numPartitions - val numTasks = stage.numTasks var emittedTaskSizeWarning = false override def toJson = { - val (taskInfoList, taskMetricsList) = taskInfos.toList.unzip - val taskInfoJson = JArray(taskInfoList.map(_.toJson)) - val taskMetricsJson = JArray(taskMetricsList.map(_.toJson)) - ("Stage ID" -> stage.id) ~ - ("Submission Time" -> submissionTime.getOrElse(0L)) ~ - ("Completion Time" -> completionTime.getOrElse(0L)) ~ - ("RDD Name" -> rddName) ~ + val (taskInfoList, taskMetricsList) = taskInfo.toList.unzip + val taskInfoListJson = JArray(taskInfoList.map(_.toJson)) + val taskMetricsListJson = JArray(taskMetricsList.map(_.toJson)) + val submissionTimeJson = submissionTime.map(JInt(_)).getOrElse(JNothing) + val completionTimeJson = completionTime.map(JInt(_)).getOrElse(JNothing) + ("Stage ID" -> stageId) ~ ("Stage Name" -> name) ~ + ("RDD Name" -> rddName) ~ ("Number of Partitions" -> numPartitions) ~ ("Number of Tasks" -> numTasks) ~ - ("Task Info" -> taskInfoJson) ~ - ("Task Metrics" -> taskMetricsJson) + ("Task Info List" -> taskInfoListJson) ~ + ("Task Metrics List" -> taskMetricsListJson) ~ + ("Submission Time" -> submissionTimeJson) ~ + ("Completion Time" -> completionTimeJson) ~ + ("Emitted Task Size Warning" -> emittedTaskSizeWarning) } } + +private[spark] +object StageInfo { + def fromStage(stage: Stage): StageInfo = { + new StageInfo( + stage.id, + stage.name, + stage.rdd.name, + stage.numPartitions, + stage.numTasks) + } + + def fromJson(json: JValue): StageInfo = { + implicit val format = DefaultFormats + val taskInfoListJson = (json \ "Task Info List").extract[List[JValue]] + val taskMetricsListJson = (json \ "Task Metrics List").extract[List[JValue]] + val taskInfo = taskInfoListJson.zip(taskMetricsListJson).map { case (info, metrics) => + (TaskInfo.fromJson(info), TaskMetrics.fromJson(metrics)) + }.toBuffer + + val metrics = new StageInfo( + (json \ "Stage ID").extract[Int], + (json \ "Stage Name").extract[String], + (json \ "RDD Name").extract[String], + (json \ "Number of Partitions").extract[Int], + (json \ "Number of Tasks").extract[Int], + taskInfo) + + metrics.submissionTime = + json \ "Submission Time" match { + case JNothing => None + case value: JValue => Some(value.extract[Long]) + } + metrics.completionTime = + json \ "Completion Time" match { + case JNothing => None + case value: JValue => Some(value.extract[Long]) + } + metrics.emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] + metrics + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6a3c9e858f0ef..14d259ccac922 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -18,6 +18,8 @@ package org.apache.spark.scheduler import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * Information about a running task attempt inside a TaskSet. @@ -95,17 +97,35 @@ class TaskInfo( def timeRunning(currentTime: Long): Long = currentTime - launchTime override def toJson = { - val _duration = if (finished) duration else 0L ("Task ID" -> taskId) ~ ("Index" -> index) ~ ("Launch Time" -> launchTime) ~ ("Executor ID" -> executorId) ~ ("Host" -> host) ~ ("Locality" -> taskLocality.toString) ~ - ("Status" -> status) ~ ("Getting Result Time" -> gettingResultTime) ~ - ("Duration" -> _duration) ~ ("Finish Time" -> finishTime) ~ - ("Failed" -> failed) + ("Failed" -> failed) ~ + ("Serialized Size" -> serializedSize) + } +} + +private[spark] +object TaskInfo { + def fromJson(json: JValue): TaskInfo = { + implicit val format = DefaultFormats + val taskInfo = new TaskInfo( + (json \ "Task ID").extract[Long], + (json \ "Index").extract[Int], + (json \ "Launch Time").extract[Long], + (json \ "Executor ID").extract[String], + (json \ "Host").extract[String], + TaskLocality.fromJson(json \ "Locality")) + + taskInfo.gettingResultTime = (json \ "Getting Result Time").extract[Long] + taskInfo.finishTime = (json \ "Finish Time").extract[Long] + taskInfo.failed = (json \ "Failed").extract[Boolean] + taskInfo.serializedSize = (json \ "Serialized Size").extract[Int] + taskInfo } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 35de13c385187..7cc39b7a65b8d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,9 +17,11 @@ package org.apache.spark.scheduler +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats private[spark] object TaskLocality extends Enumeration { - // process local is expected to be used ONLY within tasksetmanager for now. + // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value @@ -27,4 +29,19 @@ private[spark] object TaskLocality extends Enumeration { def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { condition <= constraint } + + def fromJson(json: JValue): TaskLocality = { + implicit val format = DefaultFormats + val processLocal = PROCESS_LOCAL.toString + val nodeLocal = NODE_LOCAL.toString + val rackLocal = RACK_LOCAL.toString + val any = ANY.toString + + json.extract[String] match { + case `processLocal` => PROCESS_LOCAL + case `nodeLocal` => NODE_LOCAL + case `rackLocal` => RACK_LOCAL + case `any` => ANY + } + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index afee46d74032d..ebfb3abac3039 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -23,6 +23,8 @@ import org.apache.spark.util.Utils import org.apache.spark.scheduler.JsonSerializable import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * This class represent an unique identifier for a BlockManager. @@ -90,10 +92,10 @@ private[spark] class BlockManagerId private ( } override def toJson = { - ("Executor ID" -> executorId) ~ - ("Host Port" -> hostPort) ~ - ("Host" -> host) ~ - ("Port" -> port) + ("Executor ID" -> executorId_) ~ + ("Host" -> host_) ~ + ("Port" -> port_) ~ + ("Netty Port" -> nettyPort_) } } @@ -124,4 +126,13 @@ private[spark] object BlockManagerId { blockManagerIdCache.putIfAbsent(id, id) blockManagerIdCache.get(id) } + + def fromJson(json: JValue): BlockManagerId = { + implicit val format = DefaultFormats + new BlockManagerId( + (json \ "Executor ID").extract[String], + (json \ "Host").extract[String], + (json \ "Port").extract[Int], + (json \ "Netty Port").extract[Int]) + } } 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 0196f43d7431b..fecc75e635ce7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,8 +17,6 @@ package org.apache.spark.ui -import javax.servlet.http.HttpServletRequest - import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.{Logging, SparkContext, SparkEnv} @@ -27,7 +25,10 @@ import org.apache.spark.ui.exec.ExecutorsUI import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{FileLogger, Utils} +import org.apache.spark.scheduler.{SparkListenerEvent, SparkListener} + +import net.liftweb.json.JsonAST._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { @@ -72,6 +73,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize jobs.start() + env.start() exec.start() } @@ -87,3 +89,10 @@ private[spark] object SparkUI { val DEFAULT_PORT = "4040" val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } + +private[spark] class UISparkListener(name: String) extends SparkListener { + protected val logger = new FileLogger(name) + protected def logEvent(event: SparkListenerEvent) = { + logger.logLine(compactRender(event.toJson)) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 88f41be8d3dd2..664eda045b3b0 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -25,70 +25,105 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler +import org.apache.spark.SparkContext +import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{UISparkListener, UIUtils} import org.apache.spark.ui.Page.Environment -import org.apache.spark.SparkContext - private[spark] class EnvironmentUI(sc: SparkContext) { - def getHandlers = Seq[(String, Handler)]( - ("/environment", (request: HttpServletRequest) => envDetails(request)) - ) + private var _listener: Option[EnvironmentListener] = None + def listener = _listener.get - def envDetails(request: HttpServletRequest): Seq[Node] = { - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - def jvmRow(kv: (String, String)) = {kv._1}{kv._2} - def jvmTable = - UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true) - - val sparkProperties = sc.conf.getAll.sorted - - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - - val propertyHeaders = Seq("Name", "Value") - def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val sparkPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties, fixedWidth = true) - val otherPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) - - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val classPath = (addedJars ++ addedFiles ++ classPathEntries).sorted + def start() { + _listener = Some(new EnvironmentListener) + sc.addSparkListener(listener) + } - val classPathHeaders = Seq("Resource", "Source") - def classPathRow(data: (String, String)) = {data._1}{data._2} - val classPathTable = - UIUtils.listingTable(classPathHeaders, classPathRow, classPath, fixedWidth = true) + def getHandlers = Seq[(String, Handler)]( + ("/environment", (request: HttpServletRequest) => render(request)) + ) + /** + * Render an HTML page that encodes environment information + */ + def render(request: HttpServletRequest): Seq[Node] = { + val runtimeInformationTable = UIUtils.listingTable( + propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) + val sparkPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.sparkProperties, fixedWidth = true) + val systemPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true) + val classpathEntriesTable = UIUtils.listingTable( + classPathHeaders, classPathRow, listener.classpathEntries, fixedWidth = true) val content = -

Runtime Information

{jvmTable} -

Spark Properties

- {sparkPropertyTable} -

System Properties

- {otherPropertyTable} -

Classpath Entries

- {classPathTable} +

Runtime Information

{runtimeInformationTable} +

Spark Properties

{sparkPropertiesTable} +

System Properties

{systemPropertiesTable} +

Classpath Entries

{classpathEntriesTable}
UIUtils.headerSparkPage(content, sc, "Environment", Environment) } + + private def propertyHeader = Seq("Name", "Value") + private def classPathHeaders = Seq("Resource", "Source") + private def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + private def classPathRow(data: (String, String)) = {data._1}{data._2} + + /** + * A SparkListener that logs information to be displayed on the Environment UI. + */ + private[spark] class EnvironmentListener extends UISparkListener("environment-ui") { + var jvmInformation: Seq[(String, String)] = Seq() + var sparkProperties: Seq[(String, String)] = Seq() + var systemProperties: Seq[(String, String)] = Seq() + var classpathEntries: Seq[(String, String)] = Seq() + + def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { + jvmInformation = loadEnvironment.jvmInformation + sparkProperties = loadEnvironment.sparkProperties + systemProperties = loadEnvironment.systemProperties + classpathEntries = loadEnvironment.classpathEntries + logEvent(loadEnvironment) + logger.flush() + } + + override def onJobStart(jobStart: SparkListenerJobStart) = { + logger.start() + + // Gather properties + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + val sparkProperties = sc.conf.getAll.sorted + val systemProperties = System.getProperties.iterator.toSeq + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + val classPathEntries = classPathProperty._2 + .split(sc.conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted + + // Trigger SparkListenerLoadEnvironment + val loadEnvironment = new SparkListenerLoadEnvironment( + jvmInformation, sparkProperties, otherProperties, classPaths) + onLoadEnvironment(loadEnvironment) + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() + } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 92394dea76f28..f537b531bceb8 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -19,29 +19,29 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashMap +import scala.collection.mutable import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.{ExceptionFailure, Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.{Utils, FileLogger} - -import net.liftweb.json.JsonAST._ -import net.liftweb.json.JsonDSL._ +import org.apache.spark.ui.{UISparkListener, UIUtils} +import org.apache.spark.util.Utils +import org.apache.spark.scheduler.SparkListenerTaskEnd +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler.SparkListenerTaskStart +import org.apache.spark.storage.StorageStatus private[spark] class ExecutorsUI(val sc: SparkContext) { - private val data = new ExecutorsData private var _listener: Option[ExecutorsListener] = None def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener(this)) + _listener = Some(new ExecutorsListener) sc.addSparkListener(listener) } @@ -53,43 +53,37 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { * Render an HTML page that encodes executor information */ def render(request: HttpServletRequest): Seq[Node] = { - listener.getExecutorStorageStatus() - val summaryJson = data.summaryJson - val executorsJson = data.executorIdToJson.values.toSeq - renderFromJson(summaryJson, executorsJson) - } + listener.fetchStorageStatus() + val storageStatusList = listener.storageStatusList + val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) + val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) + val execTable = UIUtils.listingTable(execHeader, execRow, execInfo) - /** - * Render an HTML page that encodes executor information from the given JSON representations - */ - def renderFromJson(summaryJson: JValue, executorsJson: Seq[JValue]): Seq[Node] = { - val maximumMemory = Utils.extractLongFromJson(summaryJson \ "Maximum Memory") - val memoryUsed = Utils.extractLongFromJson(summaryJson \ "Memory Used") - val diskSpaceUsed = Utils.extractLongFromJson(summaryJson \ "Disk Space Used") - val execTable = UIUtils.listingTable[JValue](execHeader, execRow, executorsJson) val content =
    -
  • - Memory: - {Utils.bytesToString(memoryUsed)} Used - ({Utils.bytesToString(maximumMemory)} Total) -
  • -
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
  • +
  • Memory: + {Utils.bytesToString(memUsed)} Used + ({Utils.bytesToString(maxMem)} Total)
  • +
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
-
-
- {execTable} -
-
; +
+
+ {execTable} +
+
; - UIUtils.headerSparkPage(content, sc, "Executors (" + executorsJson.size + ")", Executors) + UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) } - /** Header fields in the executors table */ + /** + * Header fields in the executors table + */ private def execHeader = Seq( "Executor ID", "Address", @@ -105,233 +99,156 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { "Shuffle Write") /** - * Render an HTML table row representing an executor from the given JSON representation + * Render an HTML table row representing an executor */ - private def execRow(executorJson: JValue): Seq[Node] = { - def getString(field: String) = Utils.extractStringFromJson(executorJson \ field) - def getLong(field: String) = Utils.extractLongFromJson(executorJson \ field) - def getInt(field: String) = Utils.extractIntFromJson(executorJson \ field) - - val maximumMemory = getLong("Maximum Memory") - val memoryUsed = getLong("Memory Used") - val diskUsed = getLong("Disk Used") - val activeTasks = getInt("Active Tasks") - val failedTasks = getInt("Failed Tasks") - val completeTasks = getInt("Complete Tasks") + private def execRow(values: Map[String, String]): Seq[Node] = { + val maximumMemory = values("Maximum Memory") + val memoryUsed = values("Memory Used") + val diskUsed = values("Disk Used") + val activeTasks = values("Active Tasks") + val failedTasks = values("Failed Tasks") + val completeTasks = values("Complete Tasks") val totalTasks = activeTasks + failedTasks + completeTasks - {getString("Executor ID")} - {getString("Address")} - {getString("RDD Blocks")} - - {Utils.bytesToString(memoryUsed)} / - {Utils.bytesToString(maximumMemory)} + {values("Executor ID")} + {values("Address")} + {values("RDD Blocks")} + + {Utils.bytesToString(memoryUsed.toLong)} / + {Utils.bytesToString(maximumMemory.toLong)} - - {Utils.bytesToString(diskUsed)} + + {Utils.bytesToString(diskUsed.toLong)} {activeTasks} {failedTasks} {completeTasks} {totalTasks} - {Utils.msDurationToString(getLong("Task Time"))} - {Utils.bytesToString(getLong("Shuffle Read"))} - {Utils.bytesToString(getLong("Shuffle Write"))} + {Utils.msDurationToString(values("Task Time").toLong)} + {Utils.bytesToString(values("Shuffle Read").toLong)} + {Utils.bytesToString(values("Shuffle Write").toLong)} } /** - * A vessel for storing executors information in JSON format. - * - * The source of data can come from two sources: (1) If the job is live, ExecutorsListener - * invokes ExecutorsData to update its data on certain Spark events. (2) If we are rendering - * the UI for a job from the past, ExecutorsData parses from these log files the necessary - * states to reconstruct the UI. + * Represent an executor's info as a map given a storage status index */ - private[spark] class ExecutorsData extends Logging { - var summaryJson: JValue = JNothing - val executorIdToJson = HashMap[String, JValue]() - - /** JSON representation of a newly discovered executor */ - private def newExecutorJson(execId: String): JValue = { - ("Executor ID" -> execId) ~ - ("Address" -> "") ~ - ("RDD Blocks" -> "") ~ - ("Maximum Memory" -> 0L) ~ - ("Memory Used" -> 0L) ~ - ("Disk Used" -> 0L) ~ - ("Active Tasks" -> 0) ~ - ("Failed Tasks" -> 0) ~ - ("Complete Tasks" -> 0) ~ - ("Task Time" -> 0L) ~ - ("Shuffle Read" -> 0L) ~ - ("Shuffle Write" -> 0L) - } - - /** - * Update the summary and per-executor storage status from JSON - */ - def storageStatusFetch(storageStatusFetchJson: JValue) { - val storageStatusList = - Utils.extractListFromJson(storageStatusFetchJson \ "Storage Status List") - - // Update summary storage information - val maximumMemory = storageStatusList.map { status => - Utils.extractLongFromJson(status \ "Maximum Memory") - }.fold(0L)(_+_) - val memoryUsed = storageStatusList.map { status => - Utils.extractLongFromJson(status \ "Memory Used") - }.fold(0L)(_+_) - val diskSpaceUsed = storageStatusList.flatMap { status => - Utils.extractListFromJson(status \ "Blocks").map { block => - Utils.extractLongFromJson(block \ "Status" \ "Disk Size") - } - }.fold(0L)(_+_) - - summaryJson = - ("Maximum Memory" -> maximumMemory) ~ - ("Memory Used" -> memoryUsed) ~ - ("Disk Space Used" -> diskSpaceUsed) - - // Update storage status for each executor - storageStatusList.foreach { status => - val execId = formatExecutorId( - Utils.extractStringFromJson(status \ "Block Manager ID" \ "Executor ID")) - if (execId != "") { - val address = Utils.extractStringFromJson(status \ "Block Manager ID" \ "Host Port") - val rddBlocks = Utils.extractListFromJson(status \ "Blocks").size - val maximumMemory = Utils.extractLongFromJson(status \ "Maximum Memory") - val memoryUsed = Utils.extractLongFromJson(status \ "Memory Used") - val diskUsed = Utils.extractLongFromJson(status \ "Disk Used") - val json = executorIdToJson.getOrElse(execId, newExecutorJson(execId)) - executorIdToJson(execId) = json.transform { - case JField("Address", _) => JField("Address", JString(address)) - case JField("RDD Blocks", _) => JField("RDD Blocks", JInt(rddBlocks)) - case JField("Maximum Memory", _) => JField("Maximum Memory", JInt(maximumMemory)) - case JField("Memory Used", _) => JField("Memory Used", JInt(memoryUsed)) - case JField("Disk Used", _) => JField("Disk Used", JInt(diskUsed)) - } - } - } - } - - /** - * Update executor information in response to a task start event - */ - def taskStart(taskStartJson: JValue) { - val execId = formatExecutorId( - Utils.extractStringFromJson(taskStartJson \ "Task Info" \ "Executor ID")) - val json = executorIdToJson.getOrElse(execId, { - // The executor ID according to the task is different from that according to SparkEnv - // This should never happen under normal circumstances... - logWarning("New executor detected during task start (%s)".format(execId)) - newExecutorJson(execId) - }) - executorIdToJson(execId) = json.transform { - case JField("Active Tasks", JInt(s)) => JField("Active Tasks", JInt(s + 1)) - } - } - - /** - * Update executor information in response to a task end event - */ - def taskEnd(taskEndJson: JValue) { - val execId = formatExecutorId( - Utils.extractStringFromJson(taskEndJson \ "Task Info" \ "Executor ID")) - val duration = Utils.extractLongFromJson(taskEndJson \ "Task Info" \ "Duration") - val taskEndReason = Utils.extractStringFromJson(taskEndJson \ "Task End Reason") - val failed = taskEndReason == ExceptionFailure.getClass.getSimpleName - val shuffleRead = Utils.extractLongFromJson( - taskEndJson \ "Task Metrics" \ "Shuffle Read Metrics" \ "Remote Bytes Read") - val shuffleWrite = Utils.extractLongFromJson( - taskEndJson \ "Task Metrics" \ "Shuffle Write Metrics" \ "Shuffle Bytes Written") - - val json = executorIdToJson.getOrElse(execId, { - // Information for this executor has vanished over the course of the task execution - // This should never happen under normal circumstances... - logWarning("New executor detected during task end (%s)".format(execId)) - newExecutorJson(execId) - }) - executorIdToJson(execId) = json.transform { - case JField("Active Tasks", JInt(t)) if t > 0 => JField("Active Tasks", JInt(t - 1)) - case JField("Failed Tasks", JInt(t)) if failed => JField("Failed Tasks", JInt(t + 1)) - case JField("Complete Tasks", JInt(t)) if !failed => JField("Complete Tasks", JInt(t + 1)) - case JField("Task Time", JInt(s)) => JField("Task Time", JInt(s + duration)) - case JField("Shuffle Read", JInt(s)) => JField("Shuffle Read", JInt(s + shuffleRead)) - case JField("Shuffle Write", JInt(s)) => JField("Shuffle Write", JInt(s + shuffleWrite)) - } - } - - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). This results in - * duplicate rows for the same executor. Thus, in this mode, we aggregate these two - * rows and use the executor ID of "" to be consistent. - */ - private def formatExecutorId(execId: String): String = { - if (execId == "localhost") "" else execId - } + private def getExecInfo(statusId: Int): Map[String, String] = { + val status = listener.storageStatusList(statusId) + val execId = status.blockManagerId.executorId + val hostPort = status.blockManagerId.hostPort + val rddBlocks = status.blocks.size + val memUsed = status.memUsed() + val maxMem = status.maxMem + val diskUsed = status.diskUsed() + val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) + val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) + val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) + val totalTasks = activeTasks + failedTasks + completedTasks + val totalDuration = listener.executorToDuration.getOrElse(execId, 0) + val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) + val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) + + // Also include fields not in the header + val execFields = execHeader ++ Seq("Maximum Memory") + + val execValues = Seq( + execId, + hostPort, + rddBlocks, + memUsed, + diskUsed, + activeTasks, + failedTasks, + completedTasks, + totalTasks, + totalDuration, + totalShuffleRead, + totalShuffleWrite + ) ++ Seq(maxMem) + + val execValuesString = execValues.map(_.toString) + + execFields.zip(execValuesString).toMap } /** - * A SparkListener that logs information to be displayed on the Executors UI. - * - * Currently, ExecutorsListener only fetches executor storage information from the driver - * on stage submit and completion. However, this is arbitrary and needs not be true. More - * specifically, each stage could be very long, in which case it would take a while before - * this information is updated and persisted to disk. An alternative approach would be to - * fetch every constant number of task events. + * A SparkListener that logs information to be displayed on the Executors UI */ - private[spark] class ExecutorsListener(ui: ExecutorsUI) extends SparkListener with Logging { - private val logger = new FileLogger("executors-ui") - - /** - * Invoke SparkEnv to ask the driver for executor storage status. This should be - * called sparingly. - */ - def getExecutorStorageStatus() = { - val storageStatusList = ui.sc.getExecutorStorageStatus - val event = SparkListenerStorageStatusFetch(storageStatusList) + private[spark] class ExecutorsListener extends UISparkListener("executors-ui") { + val executorToTasksActive = mutable.HashMap[String, Int]() + val executorToTasksComplete = mutable.HashMap[String, Int]() + val executorToTasksFailed = mutable.HashMap[String, Int]() + val executorToDuration = mutable.HashMap[String, Long]() + val executorToShuffleRead = mutable.HashMap[String, Long]() + val executorToShuffleWrite = mutable.HashMap[String, Long]() + var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus + + def fetchStorageStatus() { + val event = new SparkListenerStorageStatusFetch(sc.getExecutorStorageStatus) onStorageStatusFetch(event) } + def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { + storageStatusList = storageStatusFetch.storageStatusList + logEvent(storageStatusFetch) + } + override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() - override def onStageSubmitted(stageStart: SparkListenerStageSubmitted) = { - getExecutorStorageStatus() + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + fetchStorageStatus() logger.flush() } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { - getExecutorStorageStatus() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + fetchStorageStatus() logger.flush() } - override def onTaskStart(taskStart: SparkListenerTaskStart) = { - val eventJson = taskStart.toJson - ui.data.taskStart(eventJson) - logEvent(eventJson) + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + logEvent(taskStart) } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = { - val eventJson = taskEnd.toJson - ui.data.taskEnd(eventJson) - logEvent(eventJson) - } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val eid = formatExecutorId(taskEnd.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElseUpdate(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } - def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) = { - val eventJson = storageStatusFetch.toJson - ui.data.storageStatusFetch(eventJson) - logEvent(eventJson) - } + // Update shuffle read/write + if (taskEnd.taskMetrics != null) { + taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } - private def logEvent(event: JValue) = { - if (event != JNothing) { - logger.logLine(compactRender(event)) + taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } } + logEvent(taskEnd) + } + + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). This results in + * duplicate rows for the same executor. Thus, in this mode, we aggregate these two + * rows and use the executor ID of "" to be consistent. + */ + private def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index b9a02c576fbfb..ab82a018643ba 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -173,7 +173,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(m => m.executorRunTime).getOrElse(0) + val time = metrics.map(m => m.executorRunTime).getOrElse(0L) stageIdToTime(sid) += time totalTime += time diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b6e98942ab811..d0023d4d19663 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -217,7 +217,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val (info, metrics, exception) = taskData val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) - else metrics.map(m => m.executorRunTime).getOrElse(1) + else metrics.map(m => m.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 959d6def45817..8c2a1bd296e0e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,6 +22,7 @@ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.lang.StackTraceElement import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -40,6 +41,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.{SparkConf, SparkException, Logging} import net.liftweb.json.JsonAST._ +import net.liftweb.json.JsonDSL._ import net.liftweb.json.DefaultFormats /** @@ -47,9 +49,6 @@ import net.liftweb.json.DefaultFormats */ private[spark] object Utils extends Logging { - /** For extracting fields from JSON objects */ - implicit val format = DefaultFormats - /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -880,41 +879,65 @@ private[spark] object Utils extends Logging { /** Convert a (String, String) map to a JSON object */ def mapToJson(m: Map[String, String]): JValue = { - val jsonFields = m.map { case (k, v) => - JField(k, JString(v)) - } + val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } JObject(jsonFields.toList) } + /** Convert a JSON object to a (String, String) map */ + def mapFromJson(json: JValue): Map[String, String] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap + } + /** Convert a java Properties to a JSON object */ def propertiesToJson(properties: Properties): JValue = { Option(properties).map { p => - Utils.mapToJson(p.asScala) + mapToJson(p.asScala) }.getOrElse(JNothing) } - /** Extract a list from the given JSON AST */ - def extractListFromJson(json: JValue, default: List[JValue] = List()): List[JValue] = { - if (json == JNothing) default else json.extract[List[JValue]] + /** Convert a Json object to a java Properties */ + def propertiesFromJson(json: JValue): Properties = { + val properties = new Properties() + mapFromJson(json).map { case (k, v) => + properties.setProperty(k, v) + } + properties } - /** Extract a string from the given JSON AST */ - def extractStringFromJson(json: JValue, default: String = ""): String = { - if (json == JNothing) default else json.extract[String] + /** Convert a java stack trace to a Json object */ + def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { + JArray(stackTrace.map { case line => + ("Declaring Class" -> line.getClassName) ~ + ("Method Name" -> line.getMethodName) ~ + ("File Name" -> line.getFileName) ~ + ("Line Number" -> line.getLineNumber) + }.toList) } - /** Extract a double from the given JSON AST */ - def extractDoubleFromJson(json: JValue, default: Double = 0.0): Double = { - if (json == JNothing) default else json.extract[Double] + /** Convert a JSON object to a java stack trace */ + def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { + implicit val format = DefaultFormats + json.extract[List[JValue]].map { line => + new StackTraceElement( + (line \ "Declaring Class").extract[String], + (line \ "Method Name").extract[String], + (line \ "File Name").extract[String], + (line \ "Line Number").extract[Int]) + }.toArray } - /** Extracts a long from the given JSON AST */ - def extractLongFromJson(json: JValue, default: Long = 0L): Long = { - if (json == JNothing) default else json.extract[Long] + /** Convert an Exception to a Json object */ + def exceptionToJson(exception: Exception): JValue = { + ("Message" -> exception.toString) ~ + ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) } - /** Extract an int from the given JSON AST */ - def extractIntFromJson(json: JValue, default: Int = 0): Int = { - if (json == JNothing) default else json.extract[Int] + /** Convert a Json object to an Exception */ + def exceptionFromJson(json: JValue): Exception = { + implicit val format = DefaultFormats + val e = new Exception((json \ "Message").extract[String]) + e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) + e } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 6b9e4b3a71e53..2a0e94ab2d75a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -52,7 +52,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobID, None) val rootStage = new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None) - val rootStageInfo = new StageInfo(rootStage) + val rootStageInfo = StageInfo.fromStage(rootStage) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) joblogger.createLogWriterTest(jobID) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 4979d7dd6f786..f2be919312375 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -55,7 +55,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc first.numPartitions should be {4} first.submissionTime should be ('defined) first.completionTime should be ('defined) - first.taskInfos.length should be {4} + first.taskInfo.length should be {4} } test("StageInfo with fewer tasks than partitions") { @@ -103,20 +103,20 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.foreach { stageInfo => /* small test, so some tasks might take less than 1 millisecond, but average should be greater * than 0 ms. */ - checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") + checkNonZeroAvg(stageInfo.taskInfo.map{_._1.duration}, stageInfo + " duration") checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, + stageInfo.taskInfo.map{_._2.executorRunTime}, stageInfo + " executorRunTime") checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, + stageInfo.taskInfo.map{_._2.executorDeserializeTime}, stageInfo + " executorDeserializeTime") if (stageInfo.rddName == d4.name) { checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, + stageInfo.taskInfo.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime") } - stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => + stageInfo.taskInfo.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) From 8a2ebe6ba37b2d5efe344aa3bea343cda1411212 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Feb 2014 22:01:21 -0800 Subject: [PATCH 13/68] Fix bugs for EnvironmentUI and ExecutorsUI In particular, EnvironmentUI was not rendering until a job begins, and ExecutorsUI reports an incorrect number (format) of total tasks. --- .../apache/spark/ui/env/EnvironmentUI.scala | 32 +++++++++++-------- .../apache/spark/ui/exec/ExecutorsUI.scala | 17 ++++------ 2 files changed, 25 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 664eda045b3b0..4a7d3e2ef1df7 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -49,6 +49,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { * Render an HTML page that encodes environment information */ def render(request: HttpServletRequest): Seq[Node] = { + listener.loadEnvironment() val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( @@ -83,19 +84,10 @@ private[spark] class EnvironmentUI(sc: SparkContext) { var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() - def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { - jvmInformation = loadEnvironment.jvmInformation - sparkProperties = loadEnvironment.sparkProperties - systemProperties = loadEnvironment.systemProperties - classpathEntries = loadEnvironment.classpathEntries - logEvent(loadEnvironment) - logger.flush() - } - - override def onJobStart(jobStart: SparkListenerJobStart) = { - logger.start() - - // Gather properties + /** + * Gather JVM, spark, system and classpath properties + */ + def loadEnvironment() = { val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), ("Java Home", Properties.javaHome), @@ -124,6 +116,20 @@ private[spark] class EnvironmentUI(sc: SparkContext) { onLoadEnvironment(loadEnvironment) } + /** + * Prepare environment information for UI to render, and log the corresponding event + */ + def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { + jvmInformation = loadEnvironment.jvmInformation + sparkProperties = loadEnvironment.sparkProperties + systemProperties = loadEnvironment.systemProperties + classpathEntries = loadEnvironment.classpathEntries + logEvent(loadEnvironment) + logger.flush() + } + + override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index f537b531bceb8..a57b8b393679f 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -24,18 +24,17 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext, ExceptionFailure} import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.{UISparkListener, UIUtils} import org.apache.spark.util.Utils import org.apache.spark.scheduler.SparkListenerTaskEnd -import org.apache.spark.ExceptionFailure import org.apache.spark.scheduler.SparkListenerTaskStart import org.apache.spark.storage.StorageStatus -private[spark] class ExecutorsUI(val sc: SparkContext) { +private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { private var _listener: Option[ExecutorsListener] = None def listener = _listener.get @@ -105,10 +104,6 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maximumMemory = values("Maximum Memory") val memoryUsed = values("Memory Used") val diskUsed = values("Disk Used") - val activeTasks = values("Active Tasks") - val failedTasks = values("Failed Tasks") - val completeTasks = values("Complete Tasks") - val totalTasks = activeTasks + failedTasks + completeTasks {values("Executor ID")} @@ -121,10 +116,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {Utils.bytesToString(diskUsed.toLong)} - {activeTasks} - {failedTasks} - {completeTasks} - {totalTasks} + {values("Active Tasks")} + {values("Failed Tasks")} + {values("Complete Tasks")} + {values("Total Tasks")} {Utils.msDurationToString(values("Task Time").toLong)} {Utils.bytesToString(values("Shuffle Read").toLong)} {Utils.bytesToString(values("Shuffle Write").toLong)} From c4cd48022b3a8dbf60f458196e21ba8c9cb3b88f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Feb 2014 22:53:43 -0800 Subject: [PATCH 14/68] Also deserialize new events This includes SparkListenerLoadEnvironment and SparkListenerStorageStatusFetch --- .../spark/scheduler/SparkListener.scala | 23 +++- .../org/apache/spark/storage/BlockId.scala | 122 +++++++++++++++++- .../storage/BlockManagerMasterActor.scala | 12 ++ .../apache/spark/storage/StorageLevel.scala | 11 ++ .../apache/spark/storage/StorageUtils.scala | 21 ++- .../scala/org/apache/spark/util/Utils.scala | 22 +++- 6 files changed, 201 insertions(+), 10 deletions(-) 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 d321b7aa21c14..bf87425a594f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -147,7 +147,6 @@ object SparkListenerEvent { /** * Deserialize a SparkListenerEvent from JSON - * TODO: include newly added events! */ def fromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats @@ -159,6 +158,8 @@ object SparkListenerEvent { val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) + val loadEnvironment = Utils.getFormattedClassName(SparkListenerLoadEnvironment) + val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -169,6 +170,8 @@ object SparkListenerEvent { case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) case `shutdown` => SparkListenerShutdown + case `loadEnvironment` => loadEnvironmentFromJson(json) + case `storageStatusFetch` => storageStatusFetchFromJson(json) } } @@ -219,6 +222,22 @@ object SparkListenerEvent { (json \ "Job ID").extract[Int], JobResult.fromJson(json \ "Job Result")) } + + private def loadEnvironmentFromJson(json: JValue) = { + implicit val format = DefaultFormats + new SparkListenerLoadEnvironment( + Utils.mapFromJson(json \ "JVM Information").toSeq, + Utils.mapFromJson(json \ "Spark Properties").toSeq, + Utils.mapFromJson(json \ "System Properties").toSeq, + Utils.mapFromJson(json \ "Classpath Entries").toSeq) + } + + private def storageStatusFetchFromJson(json: JValue) = { + implicit val format = DefaultFormats + val storageStatusList = + (json \ "Storage Status List").extract[List[JValue]].map(StorageStatus.fromJson) + new SparkListenerStorageStatusFetch(storageStatusList) + } } @@ -320,7 +339,7 @@ private[spark] object StatsReportListener extends Logging { def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { val stats = d.statCounter - val quantiles = d.getQuantiles(probabilities).map{formatNumber} + val quantiles = d.getQuantiles(probabilities).map(formatNumber) logInfo(heading + stats) logInfo(percentilesHeader) logInfo("\t" + quantiles.mkString("\t")) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index b289a19b3594b..76fb57352944f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -19,8 +19,11 @@ package org.apache.spark.storage import java.util.UUID import org.apache.spark.scheduler.JsonSerializable +import org.apache.spark.util.Utils import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * Identifies a particular Block of data, usually associated with a single file. @@ -46,43 +49,89 @@ private[spark] sealed abstract class BlockId extends JsonSerializable { case _ => false } - override def toJson = ("Name" -> name) + override def toJson = "Type" -> Utils.getFormattedClassName(this) } private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex + + override def toJson = { + super.toJson ~ + ("RDD ID" -> rddId) ~ + ("Split Index" -> splitIndex) + } } private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + + override def toJson = { + super.toJson ~ + ("Shuffle ID" -> shuffleId) ~ + ("Map ID" -> mapId) ~ + ("Reduce ID" -> reduceId) + } } private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { def name = "broadcast_" + broadcastId + + override def toJson = { + super.toJson ~ + ("Broadcast ID" -> broadcastId) + } } private[spark] case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { def name = broadcastId.name + "_" + hType + + override def toJson = { + super.toJson ~ + ("Broadcast Block ID" -> broadcastId.toJson) ~ + ("Helper Type" -> hType) + } } private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { def name = "taskresult_" + taskId + + override def toJson = { + super.toJson ~ + ("Task ID" -> taskId) + } } private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId + + override def toJson = { + super.toJson ~ + ("Stream ID" -> streamId) ~ + ("Unique ID" -> uniqueId) + } } /** Id associated with temporary data managed as blocks. Not serializable. */ private[spark] case class TempBlockId(id: UUID) extends BlockId { def name = "temp_" + id + + override def toJson = { + val UUIDJson = Utils.UUIDToJson(id) + super.toJson ~ + ("Temp ID" -> UUIDJson) + } } // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id + + override def toJson = { + super.toJson ~ + ("Test ID" -> id) + } } private[spark] object BlockId { @@ -113,4 +162,75 @@ private[spark] object BlockId { case _ => throw new IllegalStateException("Unrecognized BlockId: " + id) } + + def fromJson(json: JValue): BlockId = { + implicit val format = DefaultFormats + val rddBlockId = Utils.getFormattedClassName(RDDBlockId) + val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) + val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) + val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) + val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) + val streamBlockId = Utils.getFormattedClassName(StreamBlockId) + val tempBlockId = Utils.getFormattedClassName(TempBlockId) + val testBlockId = Utils.getFormattedClassName(TestBlockId) + + (json \ "Type").extract[String] match { + case `rddBlockId` => rddBlockIdFromJson(json) + case `shuffleBlockId` => shuffleBlockIdFromJson(json) + case `broadcastBlockId` => broadcastBlockIdFromJson(json) + case `broadcastHelperBlockId` => broadcastHelperBlockIdFromJson(json) + case `taskResultBlockId` => taskResultBlockIdFromJson(json) + case `streamBlockId` => streamBlockIdFromJson(json) + case `tempBlockId` => tempBlockIdFromJson(json) + case `testBlockId` => testBlockIdFromJson(json) + } + } + + private def rddBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new RDDBlockId( + (json \ "RDD ID").extract[Int], + (json \ "Split Index").extract[Int]) + } + + private def shuffleBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new ShuffleBlockId( + (json \ "Shuffle ID").extract[Int], + (json \ "Map ID").extract[Int], + (json \ "Reduce ID").extract[Int]) + } + + private def broadcastBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new BroadcastBlockId((json \ "Broadcast ID").extract[Long]) + } + + private def broadcastHelperBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new BroadcastHelperBlockId( + broadcastBlockIdFromJson(json \ "Broadcast Block ID"), + (json \ "Helper Type").extract[String]) + } + + private def taskResultBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new TaskResultBlockId((json \ "Task ID").extract[Long]) + } + + private def streamBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new StreamBlockId( + (json \ "Stream ID").extract[Int], + (json \ "Unique ID").extract[Long]) + } + + private def tempBlockIdFromJson(json: JValue) = { + new TempBlockId(Utils.UUIDFromJson(json \ "Temp ID")) + } + + private def testBlockIdFromJson(json: JValue) = { + implicit val format = DefaultFormats + new TestBlockId((json \ "Test ID").extract[String]) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 0b600df6de143..3368cbd9237d9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -33,6 +33,8 @@ import org.apache.spark.util.{AkkaUtils, Utils} import org.apache.spark.scheduler.JsonSerializable import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -321,6 +323,16 @@ object BlockManagerMasterActor { } } + case object BlockStatus { + def fromJson(json: JValue): BlockStatus = { + implicit val format = DefaultFormats + new BlockStatus( + StorageLevel.fromJson(json \ "Storage Level"), + (json \ "Memory Size").extract[Long], + (json \ "Disk Size").extract[Long]) + } + } + class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 4b2a80b226a27..14f64c77e13d3 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,6 +21,8 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import net.liftweb.json.JsonDSL._ import org.apache.spark.scheduler.JsonSerializable +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json.DefaultFormats /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, @@ -157,4 +159,13 @@ object StorageLevel { storageLevelCache.putIfAbsent(level, level) storageLevelCache.get(level) } + + def fromJson(json: JValue): StorageLevel = { + implicit val format = DefaultFormats + new StorageLevel( + (json \ "Use Disk").extract[Boolean], + (json \ "Use Memory").extract[Boolean], + (json \ "Deserialize").extract[Boolean], + (json \ "Replication").extract[Int]) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 5dc5522fa3d9e..54be19c615ac8 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -24,6 +24,7 @@ import org.apache.spark.scheduler.JsonSerializable import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, @@ -54,13 +55,27 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, }) ("Block Manager ID" -> blockManagerId.toJson) ~ ("Maximum Memory" -> maxMem) ~ - ("Memory Used" -> memUsed) ~ - ("Memory Remaining" -> memRemaining) ~ - ("Disk Used" -> diskUsed) ~ ("Blocks" -> blocksJson) } } +private[spark] +case object StorageStatus { + def fromJson(json: JValue): StorageStatus = { + implicit val format = DefaultFormats + val blocks = (json \ "Blocks").extract[List[JValue]].map { block => + val id = BlockId.fromJson(block \ "Block ID") + val status = BlockStatus.fromJson(block \ "Status") + (id, status) + }.toMap + new StorageStatus( + BlockManagerId.fromJson(json \ "Block Manager ID"), + (json \ "Maximum Memory").extract[Long], + blocks + ) + } +} + case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) extends Ordered[RDDInfo] { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8c2a1bd296e0e..8f8d2b1aac1ba 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -896,7 +896,7 @@ private[spark] object Utils extends Logging { }.getOrElse(JNothing) } - /** Convert a Json object to a java Properties */ + /** Convert a JSON object to a java Properties */ def propertiesFromJson(json: JValue): Properties = { val properties = new Properties() mapFromJson(json).map { case (k, v) => @@ -905,7 +905,21 @@ private[spark] object Utils extends Logging { properties } - /** Convert a java stack trace to a Json object */ + /** Convert a java UUID to a JSON object */ + def UUIDToJson(id: UUID): JValue = { + ("Least Significant Bits" -> id.getLeastSignificantBits) ~ + ("Most Significant Bits" -> id.getMostSignificantBits) + } + + /** Convert a JSON object to a java UUID */ + def UUIDFromJson(json: JValue): UUID = { + implicit val format = DefaultFormats + new UUID( + (json \ "Least Significant Bits").extract[Long], + (json \ "Most Significant Bits").extract[Long]) + } + + /** Convert a java stack trace to a JSON object */ def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { JArray(stackTrace.map { case line => ("Declaring Class" -> line.getClassName) ~ @@ -927,13 +941,13 @@ private[spark] object Utils extends Logging { }.toArray } - /** Convert an Exception to a Json object */ + /** Convert an Exception to a JSON object */ def exceptionToJson(exception: Exception): JValue = { ("Message" -> exception.toString) ~ ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) } - /** Convert a Json object to an Exception */ + /** Convert a JSON object to an Exception */ def exceptionFromJson(json: JValue): Exception = { implicit val format = DefaultFormats val e = new Exception((json \ "Message").extract[String]) From d859efc34c9a5f07bae7eca7b4ab72fa19fb7e29 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 15 Feb 2014 14:01:14 -0800 Subject: [PATCH 15/68] BlockManagerUI: Add JSON functionality --- .../apache/spark/storage/StorageUtils.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 59 +++++++++++++++++- .../apache/spark/ui/env/EnvironmentUI.scala | 21 +++---- .../apache/spark/ui/exec/ExecutorsUI.scala | 51 +++------------- .../spark/ui/storage/BlockManagerUI.scala | 17 ++++-- .../apache/spark/ui/storage/IndexPage.scala | 30 ++++++---- .../org/apache/spark/ui/storage/RDDPage.scala | 60 ++++++++++++------- 7 files changed, 140 insertions(+), 102 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 54be19c615ac8..5623290112edf 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -137,8 +137,8 @@ object StorageUtils { } /* Filters storage status by a given RDD id. */ - def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int) - : Array[StorageStatus] = { + def filterStorageStatusByRDD(storageStatusList: Seq[StorageStatus], rddId: Int) + : Seq[StorageStatus] = { storageStatusList.map { status => val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus] 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 fecc75e635ce7..080d39f06abcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -26,9 +26,17 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{FileLogger, Utils} -import org.apache.spark.scheduler.{SparkListenerEvent, SparkListener} +import org.apache.spark.scheduler._ import net.liftweb.json.JsonAST._ +import org.apache.spark.storage.StorageStatus +import scala.Some +import scala.Some +import org.apache.spark.scheduler.SparkListenerStorageStatusFetch +import scala.Some +import org.apache.spark.scheduler.SparkListenerJobEnd +import org.apache.spark.scheduler.SparkListenerStageSubmitted +import org.apache.spark.scheduler.SparkListenerJobStart /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { @@ -72,6 +80,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { // DAGScheduler() requires that the port of this server is known // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + storage.start() jobs.start() env.start() exec.start() @@ -90,9 +99,57 @@ private[spark] object SparkUI { val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } +/** A SparkListener for logging events, one file per job */ private[spark] class UISparkListener(name: String) extends SparkListener { protected val logger = new FileLogger(name) + protected def logEvent(event: SparkListenerEvent) = { logger.logLine(compactRender(event.toJson)) } + + override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() +} + +/** + * A SparkListener that fetches storage information from SparkEnv and logs it as an event. + * + * The frequency at which this occurs is by default every time a stage event is triggered. + * This needs not necessarily be the case; a stage can be arbitrarily long, so any failure + * in the middle of a stage causes the storage status for that stage to be lost. + */ +private[spark] class StorageStatusFetchSparkListener( + name: String, + sc: SparkContext) + extends UISparkListener(name) { + var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus + + /** + * Fetch storage information from SparkEnv, which involves a query to the driver. This is + * expensive and should be invoked sparingly. + */ + def fetchStorageStatus() { + val storageStatus = sc.getExecutorStorageStatus + val event = new SparkListenerStorageStatusFetch(storageStatus) + onStorageStatusFetch(event) + } + + /** + * Update local state with fetch result, and log the appropriate event + */ + protected def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { + storageStatusList = storageStatusFetch.storageStatusList + logEvent(storageStatusFetch) + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + fetchStorageStatus() + logger.flush() + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + fetchStorageStatus() + logger.flush() + } } \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 4a7d3e2ef1df7..bc9bccce15e26 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -32,7 +32,6 @@ import org.apache.spark.ui.{UISparkListener, UIUtils} import org.apache.spark.ui.Page.Environment private[spark] class EnvironmentUI(sc: SparkContext) { - private var _listener: Option[EnvironmentListener] = None def listener = _listener.get @@ -45,9 +44,6 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ("/environment", (request: HttpServletRequest) => render(request)) ) - /** - * Render an HTML page that encodes environment information - */ def render(request: HttpServletRequest): Seq[Node] = { listener.loadEnvironment() val runtimeInformationTable = UIUtils.listingTable( @@ -76,7 +72,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { private def classPathRow(data: (String, String)) = {data._1}{data._2} /** - * A SparkListener that logs information to be displayed on the Environment UI. + * A SparkListener that prepares and logs information to be displayed on the Environment UI */ private[spark] class EnvironmentListener extends UISparkListener("environment-ui") { var jvmInformation: Seq[(String, String)] = Seq() @@ -84,9 +80,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() - /** - * Gather JVM, spark, system and classpath properties - */ + /** Gather JVM, spark, system and classpath properties */ def loadEnvironment() = { val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), @@ -116,9 +110,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { onLoadEnvironment(loadEnvironment) } - /** - * Prepare environment information for UI to render, and log the corresponding event - */ + /** Prepare environment information for UI to render, and log the corresponding event */ def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { jvmInformation = loadEnvironment.jvmInformation sparkProperties = loadEnvironment.sparkProperties @@ -128,8 +120,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) { logger.flush() } - override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() - - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() + override def onJobStart(jobStart: SparkListenerJobStart) = { + super.onJobStart(jobStart) + loadEnvironment() + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index a57b8b393679f..460be50f9dd66 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -25,17 +25,14 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler import org.apache.spark.{Logging, SparkContext, ExceptionFailure} -import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{UISparkListener, UIUtils} +import org.apache.spark.ui.{StorageStatusFetchSparkListener, UIUtils} import org.apache.spark.util.Utils import org.apache.spark.scheduler.SparkListenerTaskEnd import org.apache.spark.scheduler.SparkListenerTaskStart -import org.apache.spark.storage.StorageStatus private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { - private var _listener: Option[ExecutorsListener] = None def listener = _listener.get @@ -48,9 +45,6 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { ("/executors", (request: HttpServletRequest) => render(request)) ) - /** - * Render an HTML page that encodes executor information - */ def render(request: HttpServletRequest): Seq[Node] = { listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList @@ -80,9 +74,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) } - /** - * Header fields in the executors table - */ + /** Header fields for the executors table */ private def execHeader = Seq( "Executor ID", "Address", @@ -97,14 +89,11 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { "Shuffle Read", "Shuffle Write") - /** - * Render an HTML table row representing an executor - */ + /** Render an HTML row representing an executor */ private def execRow(values: Map[String, String]): Seq[Node] = { val maximumMemory = values("Maximum Memory") val memoryUsed = values("Memory Used") val diskUsed = values("Disk Used") - {values("Executor ID")} {values("Address")} @@ -126,9 +115,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { } - /** - * Represent an executor's info as a map given a storage status index - */ + /** Represent an executor's info as a map given a storage status index */ private def getExecInfo(statusId: Int): Map[String, String] = { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId @@ -169,40 +156,16 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { } /** - * A SparkListener that logs information to be displayed on the Executors UI + * A SparkListener that prepares and logs information to be displayed on the Executors UI */ - private[spark] class ExecutorsListener extends UISparkListener("executors-ui") { + private[spark] + class ExecutorsListener extends StorageStatusFetchSparkListener("executors-ui", sc) { val executorToTasksActive = mutable.HashMap[String, Int]() val executorToTasksComplete = mutable.HashMap[String, Int]() val executorToTasksFailed = mutable.HashMap[String, Int]() val executorToDuration = mutable.HashMap[String, Long]() val executorToShuffleRead = mutable.HashMap[String, Long]() val executorToShuffleWrite = mutable.HashMap[String, Long]() - var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus - - def fetchStorageStatus() { - val event = new SparkListenerStorageStatusFetch(sc.getExecutorStorageStatus) - onStorageStatusFetch(event) - } - - def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { - storageStatusList = storageStatusFetch.storageStatusList - logEvent(storageStatusFetch) - } - - override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() - - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - fetchStorageStatus() - logger.flush() - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - fetchStorageStatus() - logger.flush() - } override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = formatExecutorId(taskStart.taskInfo.executorId) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 39f422dd6b90f..05c17597e52e7 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -17,22 +17,29 @@ package org.apache.spark.ui.storage -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import org.apache.spark.{Logging, SparkContext} import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.StorageStatusFetchSparkListener /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { - val indexPage = new IndexPage(this) - val rddPage = new RDDPage(this) + private var _listener: Option[StorageStatusFetchSparkListener] = None + private val indexPage = new IndexPage(this) + private val rddPage = new RDDPage(this) + + def listener = _listener.get + + def start() { + _listener = Some(new StorageStatusFetchSparkListener("block-manager-ui", sc)) + sc.addSparkListener(listener) + } def getHandlers = Seq[(String, Handler)]( ("/storage/rdd", (request: HttpServletRequest) => rddPage.render(request)), ("/storage", (request: HttpServletRequest) => indexPage.render(request)) ) -} +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 109a7d4094c0a..8b99d9872502a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,26 +28,30 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { - val sc = parent.sc + private val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics + parent.listener.fetchStorageStatus() + val storageStatusList = parent.listener.storageStatusList - val rddHeaders = Seq( - "RDD Name", - "Storage Level", - "Cached Partitions", - "Fraction Cached", - "Size in Memory", - "Size on Disk") + // Calculate macro-level statistics val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val content = listingTable(rddHeaders, rddRow, rdds) + val content = listingTable(rddHeader, rddRow, rdds) - headerSparkPage(content, parent.sc, "Storage ", Storage) + headerSparkPage(content, sc, "Storage ", Storage) } - def rddRow(rdd: RDDInfo): Seq[Node] = { + /** Header fields for the RDD table */ + private def rddHeader = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Cached", + "Size in Memory", + "Size on Disk") + + /** Render an HTML row representing an RDD */ + private def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index b83cd54f3c39a..67f195a1e17ef 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -27,31 +27,29 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.Utils - /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { - val sc = parent.sc + private val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { + parent.listener.fetchStorageStatus() + val storageStatusList = parent.listener.storageStatusList val id = request.getParameter("id").toInt - val storageStatusList = sc.getExecutorStorageStatus val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") + // Worker table val workers = filteredStorageStatusList.map((id, _)) - val workerTable = listingTable(workerHeaders, workerRow, workers) - - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", - "Executors") + val workerTable = listingTable(workerHeader, workerRow, workers) + // Block table val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray. sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) val blocks = blockStatuses.map { case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) } - val blockTable = listingTable(blockHeaders, blockRow, blocks) + val blockTable = listingTable(blockHeader, blockRow, blocks) val content =
@@ -95,10 +93,38 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
; - headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage) + headerSparkPage(content, sc, "RDD Storage Info for " + rddInfo.name, Storage) + } + + /** Header fields for the worker table */ + private def workerHeader = Seq( + "Host", + "Memory Usage", + "Disk Usage") + + /** Header fields for the block table */ + private def blockHeader = Seq( + "Block Name", + "Storage Level", + "Size in Memory", + "Size on Disk", + "Executors") + + /** Render an HTML row representing a worker */ + private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { + val (rddId, status) = worker + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.bytesToString(status.memUsedByRDD(rddId))} + ({Utils.bytesToString(status.memRemaining)} Remaining) + + {Utils.bytesToString(status.diskUsedByRDD(rddId))} + } - def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { + /** Render an HTML row representing a block */ + private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { val (id, block, locations) = row {id} @@ -116,16 +142,4 @@ private[spark] class RDDPage(parent: BlockManagerUI) { } - - def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { - val (rddId, status) = worker - - {status.blockManagerId.host + ":" + status.blockManagerId.port} - - {Utils.bytesToString(status.memUsedByRDD(rddId))} - ({Utils.bytesToString(status.memRemaining)} Remaining) - - {Utils.bytesToString(status.diskUsedByRDD(rddId))} - - } } From 8add36bb08126fbcd02d23c446dd3ec970f1f549 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 15 Feb 2014 14:40:49 -0800 Subject: [PATCH 16/68] JobProgressUI: Add JSON functionality In addition, refactor FileLogger to log in one directory per logger --- .../spark/ui/jobs/ExecutorSummary.scala | 2 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 9 ++-- .../org/apache/spark/ui/jobs/IndexPage.scala | 16 +++---- .../spark/ui/jobs/JobProgressListener.scala | 15 +++++-- .../apache/spark/ui/jobs/JobProgressUI.scala | 7 +-- .../org/apache/spark/ui/jobs/PoolPage.scala | 13 +++--- .../org/apache/spark/ui/jobs/PoolTable.scala | 8 ++-- .../org/apache/spark/ui/jobs/StagePage.scala | 28 ++++++------ .../org/apache/spark/ui/jobs/StageTable.scala | 45 +++++++++---------- .../apache/spark/ui/storage/IndexPage.scala | 6 +-- .../org/apache/spark/ui/storage/RDDPage.scala | 5 ++- .../org/apache/spark/util/FileLogger.scala | 8 ++-- 12 files changed, 83 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 64e22a30b48f9..904ecd7e4a9d8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -/** class for reporting aggregated metrics for each executors in stageUI */ +/** Class for reporting aggregated metrics for each executors in the stage UI */ private[spark] class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index ab03eb5ce1ab4..df1e98013b301 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -24,13 +24,10 @@ import org.apache.spark.util.Utils import scala.collection.mutable /** Page showing executor summary */ -private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { +private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { + private def listener = parent.listener - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { executorTable() } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 6289f8744f240..0c44f280cc1a0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -25,10 +25,10 @@ import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils._ - /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { - def listener = parent.listener + private val sc = parent.sc + private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -47,16 +47,16 @@ private[spark] class IndexPage(parent: JobProgressUI) { parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val pools = listener.sc.getAllPools - val poolTable = new PoolTable(pools, listener) + val pools = sc.getAllPools + val poolTable = new PoolTable(pools, parent) val summary: NodeSeq = val content = summary ++ - {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { + {if (sc.getSchedulingMode == SchedulingMode.FAIR) {

{pools.size} Fair Scheduler Pools

++ poolTable.toNodeSeq } else { Seq() @@ -85,7 +85,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {

Failed Stages ({failedStages.size})

++ failedStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Stages) + headerSparkPage(content, sc, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ab82a018643ba..d5153dff7fb0b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.{ListBuffer, HashMap, HashSet} import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ +import org.apache.spark.ui.UISparkListener /** * Tracks task-level information to be displayed in the UI. @@ -30,7 +31,9 @@ import org.apache.spark.scheduler._ * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { +private[spark] class JobProgressListener(sc: SparkContext) + extends UISparkListener("job-progress-ui") { + // How many stages to remember val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) val DEFAULT_POOL_NAME = "default" @@ -60,18 +63,17 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() - override def onJobStart(jobStart: SparkListenerJobStart) {} - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo poolToActiveStages(stageIdToPool(stage.stageId)) -= stage activeStages -= stage completedStages += stage trimIfNecessary(completedStages) + logEvent(stageCompleted) } /** If stages is too large, remove and garbage collect old stages */ - def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { + private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > RETAINED_STAGES) { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { @@ -108,6 +110,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) stages += stage + logEvent(stageSubmitted) } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { @@ -118,6 +121,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) stageIdToTaskInfos(sid) = taskList + logEvent(taskStart) } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) @@ -202,6 +206,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageIdToTaskInfos(sid) = taskList + logEvent(taskEnd) } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { @@ -218,5 +223,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } case _ => } + logEvent(jobEnd) + super.onJobEnd(jobEnd) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index a0be27abad917..04525c70be1b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -30,13 +30,14 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { private var _listener: Option[JobProgressListener] = None - def listener = _listener.get - val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) + val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + + def listener = _listener.get + def start() { _listener = Some(new JobProgressListener(sc)) sc.addSparkListener(listener) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 89fffcb80d0d7..f923c8b2ea49d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -28,7 +28,8 @@ import org.apache.spark.ui.Page._ /** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { - def listener = parent.listener + private val sc = parent.sc + private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -37,13 +38,13 @@ private[spark] class PoolPage(parent: JobProgressUI) { val activeStages = poolToActiveStages.get(poolName).toSeq.flatten val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val pool = listener.sc.getPoolForName(poolName).get - val poolTable = new PoolTable(Seq(pool), listener) + val pool = sc.getPoolForName(poolName).get + val poolTable = new PoolTable(Seq(pool), parent) - val content =

Summary

++ poolTable.toNodeSeq() ++ -

{activeStages.size} Active Stages

++ activeStagesTable.toNodeSeq() + val content =

Summary

++ poolTable.toNodeSeq ++ +

{activeStages.size} Active Stages

++ activeStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Stages) + headerSparkPage(content, sc, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 22bc97ada18be..b9a6c0b975e30 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -25,11 +25,11 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { +private[spark] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { + private val poolToActiveStages = listener.poolToActiveStages + private def listener = parent.listener - var poolToActiveStages: HashMap[String, HashSet[StageInfo]] = listener.poolToActiveStages - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { poolTable(poolRow, pools) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d0023d4d19663..fccab28542f74 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,7 +23,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.{ExceptionFailure} +import org.apache.spark.ExceptionFailure import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ @@ -32,8 +32,9 @@ import org.apache.spark.scheduler.TaskInfo /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI) { - def listener = parent.listener - val dateFmt = parent.dateFmt + private val sc = parent.sc + private def dateFmt = parent.dateFmt + private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -46,7 +47,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

Summary Metrics

No tasks have started yet

Tasks

No tasks have started yet - return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Stages) + return headerSparkPage(content, sc, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageIdToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) @@ -58,12 +59,11 @@ private[spark] class StagePage(parent: JobProgressUI) { val hasShuffleWrite = shuffleWriteBytes > 0 val memoryBytesSpilled = listener.stageIdToMemoryBytesSpilled.getOrElse(stageId, 0L) val diskBytesSpilled = listener.stageIdToDiskBytesSpilled.getOrElse(stageId, 0L) - val hasBytesSpilled = (memoryBytesSpilled > 0 && diskBytesSpilled > 0) + val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) - val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished) // scalastyle:off val summary =
@@ -109,7 +109,7 @@ private[spark] class StagePage(parent: JobProgressUI) { taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (t._2.isDefined)) + val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && t._2.isDefined) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { @@ -134,9 +134,9 @@ private[spark] class StagePage(parent: JobProgressUI) { 0.0 } } - val gettingResultQuantiles = ("Time spent fetching task results" +: + val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + millis => parent.formatDuration(millis.toLong)) // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). @@ -150,9 +150,9 @@ private[spark] class StagePage(parent: JobProgressUI) { } totalExecutionTime - metrics.get.executorRunTime } - val schedulerDelayQuantiles = ("Scheduler delay" +: + val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + millis => parent.formatDuration(millis.toLong)) def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) @@ -198,15 +198,15 @@ private[spark] class StagePage(parent: JobProgressUI) { def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } - val executorTable = new ExecutorTable(parent, stageId) + val executorTable = new ExecutorTable(stageId, parent) val content = summary ++

Summary Metrics for {numCompleted} Completed Tasks

++
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++ -

Aggregated Metrics by Executor

++ executorTable.toNodeSeq() ++ +

Aggregated Metrics by Executor

++ executorTable.toNodeSeq ++

Tasks

++ taskTable - headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) + headerSparkPage(content, sc, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 999a94fc2d008..5257dbcec8d1b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -26,21 +26,20 @@ import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils - /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) { - - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR +private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { + private val sc = parent.sc + private val dateFmt = parent.dateFmt + private val isFairScheduler = sc.getSchedulingMode == SchedulingMode.FAIR + private def listener = parent.listener - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { stageTable(stageRow, stages) } } - /** Special table which merges two header cells. */ + /** Render a special table that merges two header cells. */ private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -59,21 +58,7 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr
} - private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = - { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) - -
- - {completed}/{total} {failed} - -
-
-
- } - - + /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) @@ -127,4 +112,18 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr {shuffleWrite} } + + private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = + { + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100) + +
+ + {completed}/{total} {failed} + +
+
+
+ } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 8b99d9872502a..579d11c67064f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -29,15 +29,15 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { private val sc = parent.sc + private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - parent.listener.fetchStorageStatus() - val storageStatusList = parent.listener.storageStatusList + listener.fetchStorageStatus() + val storageStatusList = listener.storageStatusList // Calculate macro-level statistics val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) val content = listingTable(rddHeader, rddRow, rdds) - headerSparkPage(content, sc, "Storage ", Storage) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 67f195a1e17ef..44db6fe5a44c1 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -30,10 +30,11 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { private val sc = parent.sc + private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - parent.listener.fetchStorageStatus() - val storageStatusList = parent.listener.storageStatusList + listener.fetchStorageStatus() + val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index d8718efa44082..74cf9f6084767 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -36,13 +36,11 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 100) { private val logDir = if (System.getenv("SPARK_LOG_DIR") != null) { - System.getenv("SPARK_LOG_DIR") + "%s/%s/".format(System.getenv("SPARK_LOG_DIR"), name) } else { - "/tmp/spark-%s".format(user) + "/tmp/spark-%s/%s/".format(user, name) } - private val logFileBase = logDir + "/" + name - private var writer: Option[PrintWriter] = { createLogDir() Some(createWriter()) // Overwrite any existing file @@ -64,7 +62,7 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 100) { /** Create a new writer to the file identified with the given path */ private def createWriter() = { - val fileWriter = new FileWriter(logFileBase + "-" + fileIndex) + val fileWriter = new FileWriter(logDir + fileIndex) val bufferedWriter = new BufferedWriter(fileWriter) new PrintWriter(bufferedWriter) } From b3976b0a2eb21b4a887d01fd16869a0f37c36f8b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 15 Feb 2014 22:52:43 -0800 Subject: [PATCH 17/68] Add functionality of reconstructing a persisted UI from SparkContext With this commit, any reconstruct SparkUI resides on default port of 14040 onwards. Logged events are posted separately from live events, such that the live SparkListeners are not affected. This commit also fixes a few JSON de/serialization bugs. --- .../scala/org/apache/spark/SparkContext.scala | 40 +++++++- .../spark/scheduler/SparkListener.scala | 3 +- .../spark/scheduler/SparkListenerBus.scala | 50 ++++++---- .../apache/spark/storage/StorageLevel.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 94 ++++++++++++++++--- .../scala/org/apache/spark/util/Utils.scala | 6 +- 6 files changed, 158 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 25f7a5ed1c250..2fe48c0b9bd64 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -24,9 +24,9 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.{Map, Set} import scala.collection.generic.Growable - import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} +import scala.io.Source import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -158,6 +158,9 @@ class SparkContext( private[spark] val ui = new SparkUI(this) ui.bind() + // Keeps track of all previously persisted UI rendered by this SparkContext + private[spark] val persistedUIs = HashMap[Int, SparkUI]() + val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -566,13 +569,45 @@ class SparkContext( .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) } - protected[spark] def checkpointFile[T: ClassTag]( path: String ): RDD[T] = { new CheckpointRDD[T](this, path) } + /** + * Render a previously persisted SparkUI from a set of event logs + * @param dirPath Path of directory containing the event logs + */ + def renderPersistedUI(dirPath: String) = { + val oldUI = new SparkUI(this, fromDisk = true) + oldUI.start() + val success = oldUI.renderFromDisk(dirPath) + if (success) { + oldUI.bind() + persistedUIs(oldUI.boundPort.get) = oldUI + } + } + + /** + * Return a list of ports bound by persisted UI's + */ + def getPersistedUIPorts = persistedUIs.keys.toSeq + + /** + * Stop the persisted UI bound to the given port, if any + */ + def stopPersistedUI(port: Int) = { + persistedUIs.remove(port).foreach(_.stop()) + } + + /** + * Stop all persisted UI's rendered in this context + */ + def stopAllPersistedUIs() = { + persistedUIs.foreach { case (port, _) => stopPersistedUI(port) } + } + /** Build the union of a list of RDDs. */ def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds) @@ -779,6 +814,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { ui.stop() + stopAllPersistedUIs() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler 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 bf87425a594f5..b21fd8bc96305 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -76,11 +76,10 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent { override def toJson = { - val _reason = Utils.getFormattedClassName(reason) super.toJson ~ ("Stage ID" -> stageId) ~ ("Task Type" -> taskType) ~ - ("Task End Reason" -> _reason) ~ + ("Task End Reason" -> reason.toJson) ~ ("Task Info" -> taskInfo.toJson) ~ ("Task Metrics" -> taskMetrics.toJson) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index b46e87f87a450..d434be7ca66fe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -40,30 +40,42 @@ private[spark] class SparkListenerBus extends Logging { override def run() { while (true) { val event = eventQueue.take - event match { - case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) - case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) - case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) - case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) - case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) - case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) - case SparkListenerShutdown => - // Get out of the while loop and shutdown the daemon thread - return - case _ => + val shutdown = postToListeners(event, sparkListeners) + if (shutdown) { + return } } } }.start() + /** + * Post an event to a given list of listeners. Return true if the shutdown event is posted. + */ + private[spark] def postToListeners(event: SparkListenerEvent, listeners: Seq[SparkListener]) + : Boolean = { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + listeners.foreach(_.onStageSubmitted(stageSubmitted)) + case stageCompleted: SparkListenerStageCompleted => + listeners.foreach(_.onStageCompleted(stageCompleted)) + case jobStart: SparkListenerJobStart => + listeners.foreach(_.onJobStart(jobStart)) + case jobEnd: SparkListenerJobEnd => + listeners.foreach(_.onJobEnd(jobEnd)) + case taskStart: SparkListenerTaskStart => + listeners.foreach(_.onTaskStart(taskStart)) + case taskGettingResult: SparkListenerTaskGettingResult => + listeners.foreach(_.onTaskGettingResult(taskGettingResult)) + case taskEnd: SparkListenerTaskEnd => + listeners.foreach(_.onTaskEnd(taskEnd)) + case SparkListenerShutdown => + // Get out of the while loop and shutdown the daemon thread + return true + case _ => + } + false + } + def addListener(listener: SparkListener) { sparkListeners += listener } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 14f64c77e13d3..59c18440da3d8 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -165,7 +165,7 @@ object StorageLevel { new StorageLevel( (json \ "Use Disk").extract[Boolean], (json \ "Use Memory").extract[Boolean], - (json \ "Deserialize").extract[Boolean], + (json \ "Deserialized").extract[Boolean], (json \ "Replication").extract[Int]) } } 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 080d39f06abcf..ba8d4b519c28f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,6 +17,10 @@ package org.apache.spark.ui +import java.io.{FileInputStream, File} + +import scala.io.Source + import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.{Logging, SparkContext, SparkEnv} @@ -27,21 +31,26 @@ import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{FileLogger, Utils} import org.apache.spark.scheduler._ - -import net.liftweb.json.JsonAST._ import org.apache.spark.storage.StorageStatus -import scala.Some -import scala.Some -import org.apache.spark.scheduler.SparkListenerStorageStatusFetch -import scala.Some -import org.apache.spark.scheduler.SparkListenerJobEnd -import org.apache.spark.scheduler.SparkListenerStageSubmitted -import org.apache.spark.scheduler.SparkListenerJobStart + +import net.liftweb.json._ +import net.liftweb.json.JsonAST.compactRender + +import it.unimi.dsi.fastutil.io.FastBufferedInputStream /** Top level user interface for Spark */ -private[spark] class SparkUI(sc: SparkContext) extends Logging { +private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + val port = if (!fromDisk) { + sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + } else { + // While each context has only one live SparkUI, it can have many persisted ones + // For persisted UI's, climb upwards from the configured / default port + val p = SparkUI.lastPersistedPort.map(_ + 1) + .getOrElse(sc.conf.get("spark.ui.persisted.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt) + SparkUI.lastPersistedPort = Some(p) + p + } var boundPort: Option[Int] = None var server: Option[Server] = None @@ -88,6 +97,65 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { def stop() { server.foreach(_.stop()) + logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + } + + /** + * Reconstruct a SparkUI previously persisted from disk from the given path. + * Return true if all required log files are found. + */ + private[spark] def renderFromDisk(dirPath: String): Boolean = { + var success = true + if (fromDisk) { + val logDir = new File(dirPath) + if (!logDir.exists || !logDir.isDirectory) { + logWarning("Given invalid directory %s when rendering persisted Spark Web UI!" + .format(dirPath)) + return false + } + val nameToListenerMap = Map[String, SparkListener]( + "job-progress-ui" -> jobs.listener, + "block-manager-ui" -> storage.listener, + "environment-ui" -> env.listener, + "executors-ui" -> exec.listener + ) + nameToListenerMap.map { case (name, listener) => + val path = "%s/%s/".format(dirPath.stripSuffix("/"), name) + val dir = new File(path) + if (dir.exists && dir.isDirectory) { + val files = dir.listFiles + Option(files).foreach { files => files.foreach(processPersistedEventLog(_, listener)) } + } else { + logWarning("%s not found when rendering persisted Spark Web UI!".format(path)) + success = false + } + } + } + success + } + + /** + * Register each event logged in the given file with the corresponding listener in order + */ + private def processPersistedEventLog(file: File, listener: SparkListener) = { + val fileStream = new FileInputStream(file) + val bufferedStream = new FastBufferedInputStream(fileStream) + var currentLine = "" + try { + val lines = Source.fromInputStream(bufferedStream).getLines() + lines.foreach { line => + currentLine = line + val listeners = Seq(listener) + val event = SparkListenerEvent.fromJson(parse(line)) + sc.dagScheduler.listenerBus.postToListeners(event, listeners) + } + } catch { + case e: Exception => + logWarning("Exception in parsing UI logs for %s".format(file.getAbsolutePath)) + logWarning(currentLine + "\n") + logDebug(e.getMessage + e.getStackTraceString) + } + bufferedStream.close() } private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") @@ -96,7 +164,11 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] object SparkUI { val DEFAULT_PORT = "4040" + val DEFAULT_PERSISTED_PORT = "14040" val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + // Keep track of the port of the last persisted UI + var lastPersistedPort: Option[Int] = None } /** A SparkListener for logging events, one file per job */ diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8f8d2b1aac1ba..9bccc8a8b64ed 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -899,8 +899,10 @@ private[spark] object Utils extends Logging { /** Convert a JSON object to a java Properties */ def propertiesFromJson(json: JValue): Properties = { val properties = new Properties() - mapFromJson(json).map { case (k, v) => - properties.setProperty(k, v) + if (json != JNothing) { + mapFromJson(json).map { case (k, v) => + properties.setProperty(k, v) + } } properties } From f3fc13b53725cdfeddcecb2068ab5a533566772f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Feb 2014 13:22:01 -0800 Subject: [PATCH 18/68] General refactor This includes reverting previous formatting and naming changes that are irrelevant to this patch. --- .../org/apache/spark/TaskEndReason.scala | 20 +-- .../apache/spark/executor/TaskMetrics.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 6 +- .../apache/spark/scheduler/JobLogger.scala | 114 +++++++++--------- .../apache/spark/scheduler/JobResult.scala | 3 +- .../spark/scheduler/SparkListener.scala | 15 +-- .../spark/scheduler/SparkListenerBus.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 8 +- .../org/apache/spark/scheduler/Task.scala | 1 + .../storage/BlockManagerMasterActor.scala | 1 + .../apache/spark/storage/StorageUtils.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 33 +++-- .../apache/spark/ui/env/EnvironmentUI.scala | 109 ++++++++--------- .../apache/spark/ui/exec/ExecutorsUI.scala | 96 +++++++-------- .../spark/ui/jobs/ExecutorSummary.scala | 2 +- .../org/apache/spark/ui/jobs/StageTable.scala | 28 ++--- .../spark/ui/storage/BlockManagerUI.scala | 9 +- .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 10 +- .../spark/scheduler/JobLoggerSuite.scala | 26 ++-- .../spark/scheduler/SparkListenerSuite.scala | 12 +- 22 files changed, 254 insertions(+), 256 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 0dbecb3ac3f15..18cd5295d15e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -31,11 +31,11 @@ import net.liftweb.json.DefaultFormats * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -sealed trait TaskEndReason extends JsonSerializable { +private[spark] sealed trait TaskEndReason extends JsonSerializable { override def toJson = "Reason" -> Utils.getFormattedClassName(this) } -case object TaskEndReason { +private[spark] case object TaskEndReason { def fromJson(json: JValue): TaskEndReason = { implicit val format = DefaultFormats val success = Utils.getFormattedClassName(Success) @@ -84,12 +84,12 @@ case object TaskEndReason { } } -case object Success extends TaskEndReason +private[spark] case object Success extends TaskEndReason // Task was finished earlier but we've now lost it -case object Resubmitted extends TaskEndReason +private[spark] case object Resubmitted extends TaskEndReason -case class FetchFailed( +private[spark] case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, @@ -104,7 +104,7 @@ case class FetchFailed( } } -case class ExceptionFailure( +private[spark] case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -125,18 +125,18 @@ case class ExceptionFailure( * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -case object TaskResultLost extends TaskEndReason +private[spark] case object TaskResultLost extends TaskEndReason -case object TaskKilled extends TaskEndReason +private[spark] case object TaskKilled extends TaskEndReason /** * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -case object ExecutorLostFailure extends TaskEndReason +private[spark] case object ExecutorLostFailure extends TaskEndReason /** * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -case object UnknownReason extends TaskEndReason +private[spark] case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 62b1bb230ef86..e6196ad554532 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -70,8 +70,8 @@ class TaskMetrics extends Serializable with JsonSerializable { var shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * If this task writes to shuffle output, metrics on the written shuffle data will be - * collected here + * If this task writes to shuffle output, metrics on the written shuffle data will be collected + * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None 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 2acfd38ac3273..847c9931f71f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util.{Utils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -596,7 +596,7 @@ class DAGScheduler( case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => val stageId = task.stageId - val taskType = task.getClass.getSimpleName + val taskType = Utils.getFormattedClassName(task) listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) @@ -826,7 +826,7 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfo += event.taskInfo -> event.taskMetrics + stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index c4ba829f1af9a..2b85e84b9992c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -34,7 +34,7 @@ import org.apache.spark.executor.TaskMetrics * is created. Note that each JobLogger only works for one SparkContext */ -class JobLogger(user: String, logDirName: String) +class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { def this() = this(System.getProperty("user.name", ""), @@ -47,9 +47,9 @@ class JobLogger(user: String, logDirName: String) "/tmp/spark-%s".format(user) } - private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIdToJobId = new HashMap[Int, Int] - private val jobIdToStageIds = new HashMap[Int, Seq[Int]] + private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIDToJobID = new HashMap[Int, Int] + private val jobIDToStageIDs = new HashMap[Int, Seq[Int]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] @@ -57,9 +57,9 @@ class JobLogger(user: String, logDirName: String) // The following 5 functions are used only in testing. private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter - private[scheduler] def getStageIdToJobId = stageIdToJobId - private[scheduler] def getJobIdToStageIds = jobIdToStageIds + private[scheduler] def getJobIDToPrintWriter = jobIDToPrintWriter + private[scheduler] def getStageIDToJobID = stageIDToJobID + private[scheduler] def getJobIDToStageIDs = jobIDToStageIDs private[scheduler] def getEventQueue = eventQueue /** Create a folder for log files, the folder's name is the creation time of jobLogger */ @@ -76,90 +76,90 @@ class JobLogger(user: String, logDirName: String) /** * Create a log file for one job - * @param jobId ID of the job + * @param jobID ID of the job * @exception FileNotFoundException Fail to create log file */ - protected def createLogWriter(jobId: Int) { + protected def createLogWriter(jobID: Int) { try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) - jobIdToPrintWriter += (jobId -> fileWriter) + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) + jobIDToPrintWriter += (jobID -> fileWriter) } catch { case e: FileNotFoundException => e.printStackTrace() } } /** - * Close log file, and clean the stage relationship in stageIdToJobId - * @param jobId ID of the job + * Close log file, and clean the stage relationship in stageIDToJobID + * @param jobID ID of the job */ - protected def closeLogWriter(jobId: Int) { - jobIdToPrintWriter.get(jobId).foreach { fileWriter => + protected def closeLogWriter(jobID: Int) { + jobIDToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() - jobIdToStageIds.get(jobId).foreach(_.foreach{ stageId => - stageIdToJobId -= stageId + jobIDToStageIDs.get(jobID).foreach(_.foreach{ stageID => + stageIDToJobID -= stageID }) - jobIdToPrintWriter -= jobId - jobIdToStageIds -= jobId + jobIDToPrintWriter -= jobID + jobIDToStageIDs -= jobID } } /** * Build up the maps that represent stage-job relationships - * @param jobId ID of the job - * @param stageIds IDs of the associated stages + * @param jobID ID of the job + * @param stageIDs IDs of the associated stages */ - protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { - jobIdToStageIds(jobId) = stageIds - stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } + protected def buildJobStageDependencies(jobID: Int, stageIDs: Seq[Int]) = { + jobIDToStageIDs(jobID) = stageIDs + stageIDs.foreach { stageID => stageIDToJobID(stageID) = jobID } } /** * Write info into log file - * @param jobId ID of the job + * @param jobID ID of the job * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { + protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + info } - jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) + jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) } /** * Write info into log file - * @param stageId ID of the stage + * @param stageID ID of the stage * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { - stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) + protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) { + stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) } /** * Log job properties into job log file - * @param jobId ID of the job + * @param jobID ID of the job * @param properties Properties of the job */ - protected def logJobProperties(jobId: Int, properties: Properties) { + protected def recordJobProperties(jobID: Int, properties: Properties) { if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobId, description, false) + jobLogInfo(jobID, description, false) } } /** * Log task metrics into job log files, including execution info and shuffle metrics - * @param stageId Stage ID of the task + * @param stageID Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info * @param taskMetrics Task running metrics */ - protected def logTaskMetrics(stageId: Int, status: String, + protected def recordTaskMetrics(stageID: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime @@ -178,7 +178,7 @@ class JobLogger(user: String, logDirName: String) case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) } /** @@ -196,8 +196,8 @@ class JobLogger(user: String, logDirName: String) * @param stageCompleted Stage completed event */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - val stageId = stageCompleted.stageInfo.stageId - stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) + val stageID = stageCompleted.stageInfo.stageId + stageLogInfo(stageID, "STAGE_ID=%d STATUS=COMPLETED".format(stageID)) } /** @@ -209,7 +209,7 @@ class JobLogger(user: String, logDirName: String) var taskStatus = taskEnd.taskType taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" - logTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) + recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) case Resubmitted => taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId @@ -223,26 +223,13 @@ class JobLogger(user: String, logDirName: String) } } - /** - * When job starts, record job property and stage graph - * @param jobStart Job start event - */ - override def onJobStart(jobStart: SparkListenerJobStart) { - val jobId = jobStart.jobId - val properties = jobStart.properties - createLogWriter(jobId) - buildJobStageDependencies(jobId, jobStart.stageIds) - logJobProperties(jobId, properties) - jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") - } - /** * When job ends, recording job completion status and close log file * @param jobEnd Job end event */ override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val jobId = jobEnd.jobId - var info = "JOB_ID=" + jobId + val jobID = jobEnd.jobId + var info = "JOB_ID=" + jobID jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -250,7 +237,20 @@ class JobLogger(user: String, logDirName: String) exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(jobId) + jobLogInfo(jobID, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(jobID) + } + + /** + * When job starts, record job property and stage graph + * @param jobStart Job start event + */ + override def onJobStart(jobStart: SparkListenerJobStart) { + val jobID = jobStart.jobId + val properties = jobStart.properties + createLogWriter(jobID) + buildJobStageDependencies(jobID, jobStart.stageIds) + recordJobProperties(jobID, properties) + jobLogInfo(jobID, "JOB_ID=" + jobID + " STATUS=STARTED") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index ab37336ebaeb8..8a94a97e675ea 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -51,8 +51,7 @@ private[spark] object JobResult { private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) - extends JobResult { +private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult { override def toJson = { val exceptionJson = Utils.exceptionToJson(exception) 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 b21fd8bc96305..6514edcf4256c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -34,9 +34,7 @@ sealed trait SparkListenerEvent extends JsonSerializable { override def toJson = "Event" -> Utils.getFormattedClassName(this) } -case class SparkListenerStageSubmitted( - stageInfo: StageInfo, - properties: Properties) +case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties) extends SparkListenerEvent { override def toJson = { val propertiesJson = Utils.propertiesToJson(properties) @@ -85,10 +83,7 @@ case class SparkListenerTaskEnd( } } -case class SparkListenerJobStart( - jobId: Int, - stageIds: Seq[Int], - properties: Properties) +case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent { override def toJson = { val stageIdsJson = JArray(stageIds.map(JInt(_)).toList) @@ -143,7 +138,6 @@ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq } object SparkListenerEvent { - /** * Deserialize a SparkListenerEvent from JSON */ @@ -279,6 +273,7 @@ trait SparkListener { * Called when a job ends */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } + } /** @@ -303,7 +298,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) // Runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfo.map{ case (info, metrics) => + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", @@ -325,7 +320,7 @@ private[spark] object StatsReportListener extends Logging { def extractDoubleDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo, TaskMetrics) => Option[Double]) : Option[Distribution] = { - Distribution(stage.stageInfo.taskInfo.flatMap { + Distribution(stage.stageInfo.taskInfos.flatMap { case ((info,metric)) => getMetric(info, metric)}) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index d434be7ca66fe..79ea2b05d0ba3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.Logging -/** Asynchronously passes SparkListenerEvent's to registered SparkListeners. */ +/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ private[spark] class SparkListenerBus extends Logging { private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener] diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c3d6dde8142b5..106ca67c5828e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -30,14 +30,13 @@ import net.liftweb.json.DefaultFormats * Stores information about a stage to pass from the scheduler to SparkListeners. Also stores the * metrics for all tasks that have completed, including redundant, speculated tasks. */ -private[spark] class StageInfo( val stageId: Int, val name: String, val rddName: String, val numPartitions: Int, val numTasks: Int, - val taskInfo: mutable.Buffer[(TaskInfo, TaskMetrics)] = + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) extends JsonSerializable { @@ -49,7 +48,7 @@ class StageInfo( var emittedTaskSizeWarning = false override def toJson = { - val (taskInfoList, taskMetricsList) = taskInfo.toList.unzip + val (taskInfoList, taskMetricsList) = taskInfos.toList.unzip val taskInfoListJson = JArray(taskInfoList.map(_.toJson)) val taskMetricsListJson = JArray(taskMetricsList.map(_.toJson)) val submissionTimeJson = submissionTime.map(JInt(_)).getOrElse(JNothing) @@ -67,7 +66,6 @@ class StageInfo( } } -private[spark] object StageInfo { def fromStage(stage: Stage): StageInfo = { new StageInfo( @@ -107,4 +105,4 @@ object StageInfo { metrics.emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] metrics } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index b85b4a50cd93a..69b42e86eae3e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream + /** * A unit of execution. We have two kinds of Task's in Spark: * - [[org.apache.spark.scheduler.ShuffleMapTask]] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 3368cbd9237d9..8465c4daacaac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -311,6 +311,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } } + private[spark] object BlockManagerMasterActor { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 5623290112edf..54be19c615ac8 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -137,8 +137,8 @@ object StorageUtils { } /* Filters storage status by a given RDD id. */ - def filterStorageStatusByRDD(storageStatusList: Seq[StorageStatus], rddId: Int) - : Seq[StorageStatus] = { + def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int) + : Array[StorageStatus] = { storageStatusList.map { status => val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus] 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 ba8d4b519c28f..14b51d8e7c5af 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -54,21 +54,24 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend var boundPort: Option[Int] = None var server: Option[Server] = None - val handlers = Seq[(String, Handler)]( + private val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("/", createRedirectHandler("/stages")) ) - val storage = new BlockManagerUI(sc) - val jobs = new JobProgressUI(sc) - val env = new EnvironmentUI(sc) - val exec = new ExecutorsUI(sc) + private val storage = new BlockManagerUI(sc) + private val jobs = new JobProgressUI(sc) + private val env = new EnvironmentUI(sc) + private val exec = new ExecutorsUI(sc) // Add MetricsServlet handlers by default - val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers + private val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ + private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers + // Listeners are not ready until SparkUI has started + private def listeners = Seq(storage.listener, jobs.listener, env.listener, exec.listener) + /** Bind the HTTP server which backs this web interface */ def bind() { try { @@ -104,7 +107,7 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend * Reconstruct a SparkUI previously persisted from disk from the given path. * Return true if all required log files are found. */ - private[spark] def renderFromDisk(dirPath: String): Boolean = { + def renderFromDisk(dirPath: String): Boolean = { var success = true if (fromDisk) { val logDir = new File(dirPath) @@ -113,14 +116,8 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend .format(dirPath)) return false } - val nameToListenerMap = Map[String, SparkListener]( - "job-progress-ui" -> jobs.listener, - "block-manager-ui" -> storage.listener, - "environment-ui" -> env.listener, - "executors-ui" -> exec.listener - ) - nameToListenerMap.map { case (name, listener) => - val path = "%s/%s/".format(dirPath.stripSuffix("/"), name) + listeners.map { listener => + val path = "%s/%s/".format(dirPath.stripSuffix("/"), listener.name) val dir = new File(path) if (dir.exists && dir.isDirectory) { val files = dir.listFiles @@ -172,7 +169,7 @@ private[spark] object SparkUI { } /** A SparkListener for logging events, one file per job */ -private[spark] class UISparkListener(name: String) extends SparkListener { +private[spark] class UISparkListener(val name: String) extends SparkListener { protected val logger = new FileLogger(name) protected def logEvent(event: SparkListenerEvent) = { @@ -224,4 +221,4 @@ private[spark] class StorageStatusFetchSparkListener( fetchStorageStatus() logger.flush() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index bc9bccce15e26..253ed3644eec1 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -36,7 +36,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { def listener = _listener.get def start() { - _listener = Some(new EnvironmentListener) + _listener = Some(new EnvironmentListener(sc)) sc.addSparkListener(listener) } @@ -70,59 +70,60 @@ private[spark] class EnvironmentUI(sc: SparkContext) { private def jvmRow(kv: (String, String)) = {kv._1}{kv._2} private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} +} + +/** + * A SparkListener that prepares and logs information to be displayed on the Environment UI + */ +private[spark] class EnvironmentListener(sc: SparkContext) + extends UISparkListener("environment-ui") { + var jvmInformation: Seq[(String, String)] = Seq() + var sparkProperties: Seq[(String, String)] = Seq() + var systemProperties: Seq[(String, String)] = Seq() + var classpathEntries: Seq[(String, String)] = Seq() + + /** Gather JVM, spark, system and classpath properties */ + def loadEnvironment() = { + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + val sparkProperties = sc.conf.getAll.sorted + val systemProperties = System.getProperties.iterator.toSeq + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + val classPathEntries = classPathProperty._2 + .split(sc.conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted + + // Trigger SparkListenerLoadEnvironment + val loadEnvironment = new SparkListenerLoadEnvironment( + jvmInformation, sparkProperties, otherProperties, classPaths) + onLoadEnvironment(loadEnvironment) + } + + /** Prepare environment information for UI to render, and log the corresponding event */ + def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { + jvmInformation = loadEnvironment.jvmInformation + sparkProperties = loadEnvironment.sparkProperties + systemProperties = loadEnvironment.systemProperties + classpathEntries = loadEnvironment.classpathEntries + logEvent(loadEnvironment) + logger.flush() + } - /** - * A SparkListener that prepares and logs information to be displayed on the Environment UI - */ - private[spark] class EnvironmentListener extends UISparkListener("environment-ui") { - var jvmInformation: Seq[(String, String)] = Seq() - var sparkProperties: Seq[(String, String)] = Seq() - var systemProperties: Seq[(String, String)] = Seq() - var classpathEntries: Seq[(String, String)] = Seq() - - /** Gather JVM, spark, system and classpath properties */ - def loadEnvironment() = { - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - val sparkProperties = sc.conf.getAll.sorted - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted - - // Trigger SparkListenerLoadEnvironment - val loadEnvironment = new SparkListenerLoadEnvironment( - jvmInformation, sparkProperties, otherProperties, classPaths) - onLoadEnvironment(loadEnvironment) - } - - /** Prepare environment information for UI to render, and log the corresponding event */ - def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { - jvmInformation = loadEnvironment.jvmInformation - sparkProperties = loadEnvironment.sparkProperties - systemProperties = loadEnvironment.systemProperties - classpathEntries = loadEnvironment.classpathEntries - logEvent(loadEnvironment) - logger.flush() - } - - override def onJobStart(jobStart: SparkListenerJobStart) = { - super.onJobStart(jobStart) - loadEnvironment() - } + override def onJobStart(jobStart: SparkListenerJobStart) = { + super.onJobStart(jobStart) + loadEnvironment() } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 460be50f9dd66..def6bbc39af2c 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.server.Handler @@ -37,7 +37,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener) + _listener = Some(new ExecutorsListener(sc)) sc.addSparkListener(listener) } @@ -154,59 +154,59 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { execFields.zip(execValuesString).toMap } +} - /** - * A SparkListener that prepares and logs information to be displayed on the Executors UI - */ - private[spark] - class ExecutorsListener extends StorageStatusFetchSparkListener("executors-ui", sc) { - val executorToTasksActive = mutable.HashMap[String, Int]() - val executorToTasksComplete = mutable.HashMap[String, Int]() - val executorToTasksFailed = mutable.HashMap[String, Int]() - val executorToDuration = mutable.HashMap[String, Long]() - val executorToShuffleRead = mutable.HashMap[String, Long]() - val executorToShuffleWrite = mutable.HashMap[String, Long]() - - override def onTaskStart(taskStart: SparkListenerTaskStart) { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - logEvent(taskStart) +/** + * A SparkListener that prepares and logs information to be displayed on the Executors UI + */ +private[spark] class ExecutorsListener(sc: SparkContext) + extends StorageStatusFetchSparkListener("executors-ui", sc) { + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + logEvent(taskStart) + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val eid = formatExecutorId(taskEnd.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = formatExecutorId(taskEnd.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElseUpdate(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + // Update shuffle read/write + if (taskEnd.taskMetrics != null) { + taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead } - // Update shuffle read/write - if (taskEnd.taskMetrics != null) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - - taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } + taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten } - logEvent(taskEnd) } + logEvent(taskEnd) + } - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). This results in - * duplicate rows for the same executor. Thus, in this mode, we aggregate these two - * rows and use the executor ID of "" to be consistent. - */ - private def formatExecutorId(execId: String): String = { - if (execId == "localhost") "" else execId - } + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). This results in + * duplicate rows for the same executor. Thus, in this mode, we aggregate these two + * rows and use the executor ID of "" to be consistent. + */ + private def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 904ecd7e4a9d8..64e22a30b48f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -/** Class for reporting aggregated metrics for each executors in the stage UI */ +/** class for reporting aggregated metrics for each executors in stageUI */ private[spark] class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5257dbcec8d1b..f5d630af31cd5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -58,6 +58,20 @@ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { } + private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = + { + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100) + +
+ + {completed}/{total} {failed} + +
+
+
+ } + /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { val submissionTime = s.submissionTime match { @@ -112,18 +126,4 @@ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { {shuffleWrite} } - - private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = - { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) - -
- - {completed}/{total} {failed} - -
-
-
- } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 05c17597e52e7..2bd77e0fa084d 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -27,14 +27,14 @@ import org.apache.spark.ui.StorageStatusFetchSparkListener /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { - private var _listener: Option[StorageStatusFetchSparkListener] = None + private var _listener: Option[BlockManagerListener] = None private val indexPage = new IndexPage(this) private val rddPage = new RDDPage(this) def listener = _listener.get def start() { - _listener = Some(new StorageStatusFetchSparkListener("block-manager-ui", sc)) + _listener = Some(new BlockManagerListener(sc)) sc.addSparkListener(listener) } @@ -42,4 +42,7 @@ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { ("/storage/rdd", (request: HttpServletRequest) => rddPage.render(request)), ("/storage", (request: HttpServletRequest) => indexPage.render(request)) ) -} \ No newline at end of file +} + +private[spark] class BlockManagerListener(sc: SparkContext) + extends StorageStatusFetchSparkListener("block-manager-ui", sc) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 579d11c67064f..8898a2dcf7e47 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -26,6 +26,7 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.Utils + /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { private val sc = parent.sc @@ -61,7 +62,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { {rdd.storageLevel.description} {rdd.numCachedPartitions} - {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} + {"%.0f".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} {Utils.bytesToString(rdd.diskSize)} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 44db6fe5a44c1..6107595d2de62 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -27,6 +27,7 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.Utils + /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { private val sc = parent.sc @@ -36,7 +37,8 @@ private[spark] class RDDPage(parent: BlockManagerUI) { listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt - val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) + val filteredStorageStatusList = + StorageUtils.filterStorageStatusByRDD(storageStatusList.toArray, id) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head // Worker table diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 74cf9f6084767..c502c83fd13cd 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -26,10 +26,10 @@ import java.util.Date * @param user User identifier if SPARK_LOG_DIR is not set, in which case log directory * defaults to /tmp/spark-[user] * @param name Name of logger, also the base name of the log files - * @param flushFrequency How many writes until the results are flushed to disk + * @param flushPeriod How many writes until the results are flushed to disk. By default, + * only flush manually */ -class FileLogger(user: String, name: String, flushFrequency: Int = 100) { - +class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALUE) { private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private var logCount = 0 private var fileIndex = 0 @@ -56,7 +56,7 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 100) { val dir = new File(logDir) if (!dir.exists && !dir.mkdirs()) { // Logger should throw a exception rather than continue to construct this object - throw new IOException("create log directory error:" + logDir) + throw new IOException("Error in creating log directory:" + logDir) } } @@ -80,7 +80,7 @@ class FileLogger(user: String, name: String, flushFrequency: Int = 100) { } writer.foreach(_.print(writeInfo)) logCount += 1 - if (logCount % flushFrequency == 0) { + if (logCount % flushPeriod == 0) { flush() logCount = 0 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 2a0e94ab2d75a..b13dc1c56ad6d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -56,21 +56,21 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) joblogger.createLogWriterTest(jobID) - joblogger.getJobIdToPrintWriter.size should be (1) - joblogger.getJobIdToStageIds.get(jobID).get.size should be (2) - joblogger.getStageIdToJobId.get(0) should be (Some(jobID)) - joblogger.getStageIdToJobId.get(1) should be (Some(jobID)) + joblogger.getJobIDToPrintWriter.size should be (1) + joblogger.getJobIDToStageIDs.get(jobID).get.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) + joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) joblogger.closeLogWriterTest(jobID) - joblogger.getStageIdToJobId.size should be (0) - joblogger.getJobIdToStageIds.size should be (0) - joblogger.getJobIdToPrintWriter.size should be (0) + joblogger.getStageIDToJobID.size should be (0) + joblogger.getJobIDToStageIDs.size should be (0) + joblogger.getJobIDToPrintWriter.size should be (0) } test("inner variables") { sc = new SparkContext("local[4]", "joblogger") val joblogger = new JobLogger { override protected def closeLogWriter(jobID: Int) = - getJobIdToPrintWriter.get(jobID).foreach { fileWriter => + getJobIDToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() } } @@ -83,11 +83,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIdToPrintWriter.size should be (1) - joblogger.getStageIdToJobId.size should be (2) - joblogger.getStageIdToJobId.get(0) should be (Some(0)) - joblogger.getStageIdToJobId.get(1) should be (Some(0)) - joblogger.getJobIdToStageIds.size should be (1) + joblogger.getJobIDToPrintWriter.size should be (1) + joblogger.getStageIDToJobID.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(0)) + joblogger.getStageIDToJobID.get(1) should be (Some(0)) + joblogger.getJobIDToStageIDs.size should be (1) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index f2be919312375..a34976af44963 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -55,7 +55,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc first.numPartitions should be {4} first.submissionTime should be ('defined) first.completionTime should be ('defined) - first.taskInfo.length should be {4} + first.taskInfos.length should be {4} } test("StageInfo with fewer tasks than partitions") { @@ -103,20 +103,20 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.foreach { stageInfo => /* small test, so some tasks might take less than 1 millisecond, but average should be greater * than 0 ms. */ - checkNonZeroAvg(stageInfo.taskInfo.map{_._1.duration}, stageInfo + " duration") + checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") checkNonZeroAvg( - stageInfo.taskInfo.map{_._2.executorRunTime}, + stageInfo.taskInfos.map{_._2.executorRunTime}, stageInfo + " executorRunTime") checkNonZeroAvg( - stageInfo.taskInfo.map{_._2.executorDeserializeTime}, + stageInfo.taskInfos.map{_._2.executorDeserializeTime}, stageInfo + " executorDeserializeTime") if (stageInfo.rddName == d4.name) { checkNonZeroAvg( - stageInfo.taskInfo.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, + stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime") } - stageInfo.taskInfo.foreach { case (taskInfo, taskMetrics) => + stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) From 3fd584e30aaf6552179bf9e9b350b130fa92d0ad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Feb 2014 18:01:12 -0800 Subject: [PATCH 19/68] Fix two major bugs First, JobProgessListener uses HashSets of TaskInfo and StageInfo, and relies on the equality of these objects to remove from the corresponding HashSets correctly. This is not a luxury that deserialized StageInfo's and TaskInfo's have. Instead, when removing from these collections, we must match by the ID rather than the object itself. Second, although SparkUI differentiates between persisted and live UI's, its children UI's and their corresponding listeners do not. Thus, each revived UI essentially duplicated all the logs that reconstructed it in the first place. Further, these zombie UI's continued to respond to live SparkListenerEvents. This has been fixed by requiring that revived UI's do not register their listeners with the current SparkContext. With the former fix, there were major incompatibility issues with the existing way UI classes access and mutate the collections. Formatting improvements associated with smoothing out these inconsistencies are included as part of this commit. --- .../spark/scheduler/SparkListener.scala | 10 + .../apache/spark/scheduler/StageInfo.scala | 1 - .../scala/org/apache/spark/ui/SparkUI.scala | 48 ++-- .../apache/spark/ui/env/EnvironmentUI.scala | 17 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 18 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 13 +- .../spark/ui/jobs/JobProgressListener.scala | 68 +++--- .../apache/spark/ui/jobs/JobProgressUI.scala | 12 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 7 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 9 +- .../org/apache/spark/ui/jobs/StagePage.scala | 209 +++++++++--------- .../org/apache/spark/ui/jobs/StageTable.scala | 53 +++-- .../spark/ui/storage/BlockManagerUI.scala | 14 +- .../apache/spark/ui/storage/IndexPage.scala | 2 +- .../org/apache/spark/util/FileLogger.scala | 6 +- 15 files changed, 257 insertions(+), 230 deletions(-) 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 6514edcf4256c..78bb61c51fd1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -274,6 +274,16 @@ trait SparkListener { */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } + /** + * Called when the Spark environment is loaded + */ + def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { } + + /** + * Called when Spark fetches storage statuses from the driver + */ + def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { } + } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 106ca67c5828e..621e8a5c81ed3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -23,7 +23,6 @@ import org.apache.spark.executor.TaskMetrics import net.liftweb.json.JsonAST._ import net.liftweb.json.JsonDSL._ -import org.apache.spark.util.Utils import net.liftweb.json.DefaultFormats /** 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 14b51d8e7c5af..c74f8b4153c13 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -58,10 +58,10 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("/", createRedirectHandler("/stages")) ) - private val storage = new BlockManagerUI(sc) - private val jobs = new JobProgressUI(sc) - private val env = new EnvironmentUI(sc) - private val exec = new ExecutorsUI(sc) + private val storage = new BlockManagerUI(sc, fromDisk) + private val jobs = new JobProgressUI(sc, fromDisk) + private val env = new EnvironmentUI(sc, fromDisk) + private val exec = new ExecutorsUI(sc, fromDisk) // Add MetricsServlet handlers by default private val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers @@ -142,9 +142,8 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend val lines = Source.fromInputStream(bufferedStream).getLines() lines.foreach { line => currentLine = line - val listeners = Seq(listener) val event = SparkListenerEvent.fromJson(parse(line)) - sc.dagScheduler.listenerBus.postToListeners(event, listeners) + sc.dagScheduler.listenerBus.postToListeners(event, Seq(listener)) } } catch { case e: Exception => @@ -169,16 +168,25 @@ private[spark] object SparkUI { } /** A SparkListener for logging events, one file per job */ -private[spark] class UISparkListener(val name: String) extends SparkListener { - protected val logger = new FileLogger(name) +private[spark] class UISparkListener(val name: String, fromDisk: Boolean = false) + extends SparkListener with Logging { + protected val logger: Option[FileLogger] = if (!fromDisk) { + Some(new FileLogger(name)) + } else { + None + } protected def logEvent(event: SparkListenerEvent) = { - logger.logLine(compactRender(event.toJson)) + // Log events only if the corresponding UI is not rendered from disk + if (!fromDisk) { + logWarning("Logging %s".format(Utils.getFormattedClassName(event))) + logger.foreach(_.logLine(compactRender(event.toJson))) + } } - override def onJobStart(jobStart: SparkListenerJobStart) = logger.start() + override def onJobStart(jobStart: SparkListenerJobStart) = logger.foreach(_.start()) - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.close() + override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.foreach(_.close()) } /** @@ -190,8 +198,9 @@ private[spark] class UISparkListener(val name: String) extends SparkListener { */ private[spark] class StorageStatusFetchSparkListener( name: String, - sc: SparkContext) - extends UISparkListener(name) { + sc: SparkContext, + fromDisk: Boolean = false) + extends UISparkListener(name, fromDisk) { var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus /** @@ -207,18 +216,13 @@ private[spark] class StorageStatusFetchSparkListener( /** * Update local state with fetch result, and log the appropriate event */ - protected def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { + override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { storageStatusList = storageStatusFetch.storageStatusList logEvent(storageStatusFetch) + logger.foreach(_.flush()) } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - fetchStorageStatus() - logger.flush() - } + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = fetchStorageStatus() - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - fetchStorageStatus() - logger.flush() - } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = fetchStorageStatus() } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 253ed3644eec1..620af21b7bd70 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -31,13 +31,15 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.{UISparkListener, UIUtils} import org.apache.spark.ui.Page.Environment -private[spark] class EnvironmentUI(sc: SparkContext) { +private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[EnvironmentListener] = None def listener = _listener.get def start() { - _listener = Some(new EnvironmentListener(sc)) - sc.addSparkListener(listener) + _listener = Some(new EnvironmentListener(sc, fromDisk)) + if (!fromDisk) { + sc.addSparkListener(listener) + } } def getHandlers = Seq[(String, Handler)]( @@ -75,8 +77,8 @@ private[spark] class EnvironmentUI(sc: SparkContext) { /** * A SparkListener that prepares and logs information to be displayed on the Environment UI */ -private[spark] class EnvironmentListener(sc: SparkContext) - extends UISparkListener("environment-ui") { +private[spark] class EnvironmentListener(sc: SparkContext, fromDisk: Boolean = false) + extends UISparkListener("environment-ui", fromDisk) { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() @@ -113,17 +115,16 @@ private[spark] class EnvironmentListener(sc: SparkContext) } /** Prepare environment information for UI to render, and log the corresponding event */ - def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { + override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { jvmInformation = loadEnvironment.jvmInformation sparkProperties = loadEnvironment.sparkProperties systemProperties = loadEnvironment.systemProperties classpathEntries = loadEnvironment.classpathEntries logEvent(loadEnvironment) - logger.flush() + logger.foreach(_.flush()) } override def onJobStart(jobStart: SparkListenerJobStart) = { - super.onJobStart(jobStart) loadEnvironment() } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index def6bbc39af2c..7eb75e8bf93ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.{Logging, SparkContext, ExceptionFailure} +import org.apache.spark.{SparkContext, ExceptionFailure} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.{StorageStatusFetchSparkListener, UIUtils} @@ -32,13 +32,15 @@ import org.apache.spark.util.Utils import org.apache.spark.scheduler.SparkListenerTaskEnd import org.apache.spark.scheduler.SparkListenerTaskStart -private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { +private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[ExecutorsListener] = None def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener(sc)) - sc.addSparkListener(listener) + _listener = Some(new ExecutorsListener(sc, fromDisk)) + if (!fromDisk) { + sc.addSparkListener(listener) + } } def getHandlers = Seq[(String, Handler)]( @@ -46,7 +48,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { ) def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() + if (!fromDisk) { + listener.fetchStorageStatus() + } val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) @@ -159,8 +163,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) extends Logging { /** * A SparkListener that prepares and logs information to be displayed on the Executors UI */ -private[spark] class ExecutorsListener(sc: SparkContext) - extends StorageStatusFetchSparkListener("executors-ui", sc) { +private[spark] class ExecutorsListener(sc: SparkContext, fromDisk: Boolean = false) + extends StorageStatusFetchSparkListener("executors-ui", sc, fromDisk) { val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 0c44f280cc1a0..77fb8c1105616 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -26,22 +26,17 @@ import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils._ /** Page showing list of all ongoing and recently finished stages and pools*/ -private[spark] class IndexPage(parent: JobProgressUI) { +private[spark] class IndexPage(parent: JobProgressUI, fromDisk: Boolean = false) { private val sc = parent.sc private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val activeStages = listener.activeStages.toSeq + val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - var activeTime = 0L - for (tasks <- listener.stageIdToTasksActive.values; t <- tasks) { - activeTime += t.timeRunning(now) - } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) @@ -79,9 +74,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { Seq() }} ++

Active Stages ({activeStages.size})

++ - activeStagesTable.toNodeSeq++ + activeStagesTable.toNodeSeq ++

Completed Stages ({completedStages.size})

++ - completedStagesTable.toNodeSeq++ + completedStagesTable.toNodeSeq ++

Failed Stages ({failedStages.size})

++ failedStagesTable.toNodeSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index d5153dff7fb0b..21aee456fb122 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -31,8 +31,8 @@ import org.apache.spark.ui.UISparkListener * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(sc: SparkContext) - extends UISparkListener("job-progress-ui") { +private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = false) + extends UISparkListener("job-progress-ui", fromDisk) { // How many stages to remember val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) @@ -40,9 +40,9 @@ private[spark] class JobProgressListener(sc: SparkContext) val stageIdToPool = new HashMap[Int, String]() val stageIdToDescription = new HashMap[Int, String]() - val poolToActiveStages = new HashMap[String, HashSet[StageInfo]]() + val poolToActiveStages = new HashMap[String, HashMap[Int, StageInfo]]() - val activeStages = HashSet[StageInfo]() + val activeStages = HashMap[Int, StageInfo]() val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() @@ -56,17 +56,18 @@ private[spark] class JobProgressListener(sc: SparkContext) val stageIdToShuffleWrite = HashMap[Int, Long]() val stageIdToMemoryBytesSpilled = HashMap[Int, Long]() val stageIdToDiskBytesSpilled = HashMap[Int, Long]() - val stageIdToTasksActive = HashMap[Int, HashSet[TaskInfo]]() + val stageIdToTasksActive = HashMap[Int, HashMap[Long, TaskInfo]]() val stageIdToTasksComplete = HashMap[Int, Int]() val stageIdToTasksFailed = HashMap[Int, Int]() - val stageIdToTaskInfos = - HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + val stageIdToTaskInfos = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo - poolToActiveStages(stageIdToPool(stage.stageId)) -= stage - activeStages -= stage + val stageId = stage.stageId + // Remove by stageId, rather than by StageInfo, in case the StageInfo is persisted + poolToActiveStages(stageIdToPool(stageId)).remove(stageId) + activeStages.remove(stageId) completedStages += stage trimIfNecessary(completedStages) logEvent(stageCompleted) @@ -96,7 +97,7 @@ private[spark] class JobProgressListener(sc: SparkContext) /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { val stage = stageSubmitted.stageInfo - activeStages += stage + activeStages(stage.stageId) = stage val poolName = Option(stageSubmitted.properties).map { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) @@ -108,19 +109,19 @@ private[spark] class JobProgressListener(sc: SparkContext) } description.map(d => stageIdToDescription(stage.stageId) = d) - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) - stages += stage + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) + stages(stage.stageId) = stage logEvent(stageSubmitted) } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val sid = taskStart.stageId - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive += taskStart.taskInfo - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskStart.taskInfo, None, None)) - stageIdToTaskInfos(sid) = taskList + val taskInfo = taskStart.taskInfo + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + tasksActive(taskInfo.taskId) = taskInfo + val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) + stageIdToTaskInfos(sid) = taskMap logEvent(taskStart) } @@ -163,8 +164,9 @@ private[spark] class JobProgressListener(sc: SparkContext) case _ => {} } - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive -= taskEnd.taskInfo + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is persisted + tasksActive.remove(taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { @@ -201,29 +203,31 @@ private[spark] class JobProgressListener(sc: SparkContext) val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) stageIdToDiskBytesSpilled(sid) += diskBytesSpilled - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList -= ((taskEnd.taskInfo, None, None)) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageIdToTaskInfos(sid) = taskList + val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + val taskInfo = taskEnd.taskInfo + taskMap(taskInfo.taskId) = new TaskUIData(taskInfo, metrics, failureInfo) + stageIdToTaskInfos(sid) = taskMap logEvent(taskEnd) } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { jobEnd.jobResult match { case JobFailed(_, stageId) => - // If two jobs share a stage we could get this failure message twice. - // So we first check whether we've already retired this stage. - val stageInfo = activeStages.filter(s => s.stageId == stageId).headOption - stageInfo.foreach {s => - activeStages -= s - poolToActiveStages(stageIdToPool(stageId)) -= s + activeStages.get(stageId).foreach { s => + // Remove by stageId, rather than by StageInfo, in case the StageInfo is persisted + activeStages.remove(s.stageId) + poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) failedStages += s trimIfNecessary(failedStages) } case _ => } logEvent(jobEnd) - super.onJobEnd(jobEnd) + logger.foreach(_.close()) } } + +private[spark] case class TaskUIData( + taskInfo: TaskInfo, + taskMetrics: Option[TaskMetrics] = None, + exception: Option[ExceptionFailure] = None) \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 04525c70be1b7..d4e01bbc9d0b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -28,10 +28,10 @@ import org.apache.spark.SparkContext import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(val sc: SparkContext) { +private[spark] class JobProgressUI(val sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[JobProgressListener] = None - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) + private val indexPage = new IndexPage(this, fromDisk) + private val stagePage = new StagePage(this, fromDisk) private val poolPage = new PoolPage(this) val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") @@ -39,8 +39,10 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get def start() { - _listener = Some(new JobProgressListener(sc)) - sc.addSparkListener(listener) + _listener = Some(new JobProgressListener(sc, fromDisk)) + if (!fromDisk) { + sc.addSparkListener(listener) + } } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index f923c8b2ea49d..bd78273c8dc01 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -27,7 +27,7 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI) { +private[spark] class PoolPage(parent: JobProgressUI, fromDisk: Boolean = false) { private val sc = parent.sc private def listener = parent.listener @@ -35,7 +35,10 @@ private[spark] class PoolPage(parent: JobProgressUI) { listener.synchronized { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages - val activeStages = poolToActiveStages.get(poolName).toSeq.flatten + val activeStages = poolToActiveStages.get(poolName) match { + case Some(s) => s.values.toSeq + case None => Seq() + } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val pool = sc.getPoolForName(poolName).get diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index b9a6c0b975e30..2329bba779462 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -35,9 +35,10 @@ private[spark] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { } } - private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[StageInfo]]) => Seq[Node], - rows: Seq[Schedulable] - ): Seq[Node] = { + private def poolTable( + makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], + rows: Seq[Schedulable]) + : Seq[Node] = { @@ -53,7 +54,7 @@ private[spark] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) {
Pool Name
} - private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[StageInfo]]) + private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]) : Seq[Node] = { val activeStages = poolToActiveStages.get(p.name) match { case Some(stages) => stages.size diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index fccab28542f74..0d876999f2758 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.scheduler.TaskInfo /** Page showing statistics and task list for a given stage */ -private[spark] class StagePage(parent: JobProgressUI) { +private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) { private val sc = parent.sc private def dateFmt = parent.dateFmt private def listener = parent.listener @@ -39,7 +39,6 @@ private[spark] class StagePage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt - val now = System.currentTimeMillis() if (!listener.stageIdToTaskInfos.contains(stageId)) { val content = @@ -50,9 +49,9 @@ private[spark] class StagePage(parent: JobProgressUI) { return headerSparkPage(content, sc, "Details for Stage %s".format(stageId), Stages) } - val tasks = listener.stageIdToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) + val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = tasks.count(_._1.finished) + val numCompleted = tasks.count(_.taskInfo.finished) val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) val hasShuffleRead = shuffleReadBytes > 0 val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L) @@ -62,7 +61,9 @@ private[spark] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) + val now = System.currentTimeMillis() + val tasksActive = listener.stageIdToTasksActive(stageId).values + tasksActive.foreach(activeTime += _.timeRunning(now)) // scalastyle:off val summary = @@ -109,25 +110,27 @@ private[spark] class StagePage(parent: JobProgressUI) { taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && t._2.isDefined) + val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { None } else { - val serializationTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.resultSerializationTime.toDouble} + val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.resultSerializationTime.toDouble + } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) - val serviceTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.executorRunTime.toDouble} - val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) + val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.executorRunTime.toDouble + } + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() + .map(ms => parent.formatDuration(ms.toLong)) - val gettingResultTimes = validTasks.map{case (info, metrics, exception) => + val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { (info.finishTime - info.gettingResultTime).toDouble } else { @@ -135,12 +138,12 @@ private[spark] class StagePage(parent: JobProgressUI) { } } val gettingResultQuantiles = "Time spent fetching task results" +: - Distribution(gettingResultTimes).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong)) + Distribution(gettingResultTimes).get.getQuantiles() + .map(millis => parent.formatDuration(millis.toLong)) // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). - val schedulerDelays = validTasks.map{case (info, metrics, exception) => + val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => val totalExecutionTime = { if (info.gettingResultTime > 0) { (info.gettingResultTime - info.launchTime).toDouble @@ -151,34 +154,30 @@ private[spark] class StagePage(parent: JobProgressUI) { totalExecutionTime - metrics.get.executorRunTime } val schedulerDelayQuantiles = "Scheduler delay" +: - Distribution(schedulerDelays).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong)) + Distribution(schedulerDelays).get.getQuantiles() + .map(millis => parent.formatDuration(millis.toLong)) def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) - val shuffleReadSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) - val shuffleWriteSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - val memoryBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.memoryBytesSpilled.toDouble + val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = "Shuffle spill (memory)" +: getQuantileCols(memoryBytesSpilledSizes) - val diskBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.diskBytesSpilled.toDouble + val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = "Shuffle spill (disk)" +: getQuantileCols(diskBytesSpilledSizes) @@ -211,83 +210,85 @@ private[spark] class StagePage(parent: JobProgressUI) { } def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean, bytesSpilled: Boolean) - (taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { + (taskData: TaskUIData): Seq[Node] = { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) - val (info, metrics, exception) = taskData - - val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) - else metrics.map(m => m.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") - val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) - val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L) - - val maybeShuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") - - val maybeShuffleWrite = - metrics.flatMap{m => m.shuffleWriteMetrics}.map(s => s.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") - - val maybeWriteTime = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleWriteTime) - val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map{ ms => - if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("") - - val maybeMemoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled) - val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") - val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") - - val maybeDiskBytesSpilled = metrics.map{m => m.diskBytesSpilled} - val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") - val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") - - - {info.index} - {info.taskId} - {info.status} - {info.taskLocality} - {info.host} - {dateFmt.format(new Date(info.launchTime))} - - {formatDuration} - - - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - - {if (shuffleRead) { - - {shuffleReadReadable} - - }} - {if (shuffleWrite) { - - {writeTimeReadable} - - - {shuffleWriteReadable} - - }} - {if (bytesSpilled) { - - {memoryBytesSpilledReadable} + + taskData match { case TaskUIData(info, metrics, exception) => + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(_.executorRunTime).getOrElse(1L) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + + val maybeShuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + + val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else parent.formatDuration(ms) + }.getOrElse("") + + val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) + val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") + val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") + + val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled) + val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") + val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") + + + {info.index} + {info.taskId} + {info.status} + {info.taskLocality} + {info.host} + {dateFmt.format(new Date(info.launchTime))} + + {formatDuration} + + + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - {diskBytesSpilledReadable} + + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - }} - {exception.map(e => - - {e.className} ({e.description})
- {fmtStackTrace(e.stackTrace)} -
).getOrElse("")} - - + {if (shuffleRead) { + + {shuffleReadReadable} + + }} + {if (shuffleWrite) { + + {writeTimeReadable} + + + {shuffleWriteReadable} + + }} + {if (bytesSpilled) { + + {memoryBytesSpilledReadable} + + + {diskBytesSpilledReadable} + + }} + {exception.map(e => + + {e.className} ({e.description})
+ {fmtStackTrace(e.stackTrace)} +
).getOrElse("")} + + + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index f5d630af31cd5..0831ea7084a69 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import java.util.Date import scala.xml.Node -import scala.collection.mutable.HashSet +import scala.collection.mutable.HashMap import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils @@ -74,51 +74,48 @@ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) + val nameLink = + {s.name} + val description = listener.stageIdToDescription.get(s.stageId) + .map(d =>
{d}
{nameLink}
).getOrElse(nameLink) val submissionTime = s.submissionTime match { - case Some(t) => dateFmt.format(new Date(t)) + case Some(d) => dateFmt.format(new Date(d)) case None => "Unknown" } - + val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) + val duration = s.submissionTime.map(t => finishTime - t) + val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val startedTasks = + listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size + val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) + val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => "" + } + val totalTasks = s.numTasks val shuffleReadSortable = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L) val shuffleRead = shuffleReadSortable match { case 0 => "" case b => Utils.bytesToString(b) } - val shuffleWriteSortable = listener.stageIdToShuffleWrite.getOrElse(s.stageId, 0L) val shuffleWrite = shuffleWriteSortable match { case 0 => "" case b => Utils.bytesToString(b) } - - val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashSet[TaskInfo]()).size - val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) - val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => "" - } - val totalTasks = s.numTasks - - val poolName = listener.stageIdToPool.get(s.stageId) - - val nameLink = - {s.name} - val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
{d}
{nameLink}
).getOrElse(nameLink) - val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) - val duration = s.submissionTime.map(t => finishTime - t) - {s.stageId} {if (isFairScheduler) { - - {poolName.get}} - } + + + {poolName.get} + + + }} {description} {submissionTime} - - {duration.map(d => parent.formatDuration(d)).getOrElse("Unknown")} - + {formattedDuration} {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 2bd77e0fa084d..bbd75164c1516 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -21,12 +21,12 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.StorageStatusFetchSparkListener /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { +private[spark] class BlockManagerUI(val sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[BlockManagerListener] = None private val indexPage = new IndexPage(this) private val rddPage = new RDDPage(this) @@ -34,8 +34,10 @@ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { def listener = _listener.get def start() { - _listener = Some(new BlockManagerListener(sc)) - sc.addSparkListener(listener) + _listener = Some(new BlockManagerListener(sc, fromDisk)) + if (!fromDisk) { + sc.addSparkListener(listener) + } } def getHandlers = Seq[(String, Handler)]( @@ -44,5 +46,5 @@ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { ) } -private[spark] class BlockManagerListener(sc: SparkContext) - extends StorageStatusFetchSparkListener("block-manager-ui", sc) +private[spark] class BlockManagerListener(sc: SparkContext, fromDisk: Boolean = false) + extends StorageStatusFetchSparkListener("block-manager-ui", sc, fromDisk) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 8898a2dcf7e47..19e173cf758da 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -62,7 +62,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { {rdd.storageLevel.description} {rdd.numCachedPartitions} - {"%.0f".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} + {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} {Utils.bytesToString(rdd.diskSize)} diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index c502c83fd13cd..f25a33c73989b 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import java.io._ import java.text.SimpleDateFormat import java.util.Date +import org.apache.spark.Logging /** * A generic class for logging information to file @@ -29,7 +30,7 @@ import java.util.Date * @param flushPeriod How many writes until the results are flushed to disk. By default, * only flush manually */ -class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALUE) { +class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALUE) extends Logging { private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private var logCount = 0 private var fileIndex = 0 @@ -54,6 +55,9 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU /** Create a logging directory with the given path */ private def createLogDir() = { val dir = new File(logDir) + if (dir.exists) { + logWarning("Logging directory already exists: " + logDir) + } if (!dir.exists && !dir.mkdirs()) { // Logger should throw a exception rather than continue to construct this object throw new IOException("Error in creating log directory:" + logDir) From 5ac906d4dfd546c5d6b6e80540c8774f3985fecc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Feb 2014 21:38:16 -0800 Subject: [PATCH 20/68] Mostly naming, formatting, and code style changes --- .../scala/org/apache/spark/SparkContext.scala | 13 +- .../org/apache/spark/TaskEndReason.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../apache/spark/scheduler/JobLogger.scala | 32 ++-- .../apache/spark/scheduler/JobResult.scala | 13 +- .../spark/scheduler/SparkListener.scala | 32 ++-- .../spark/scheduler/SparkListenerBus.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 14 +- .../org/apache/spark/storage/BlockId.scala | 1 + .../apache/spark/storage/StorageLevel.scala | 3 +- .../apache/spark/storage/StorageUtils.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 24 +-- .../apache/spark/ui/env/EnvironmentUI.scala | 1 + .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 13 +- .../spark/ui/jobs/JobProgressListener.scala | 10 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 5 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 1 - .../org/apache/spark/ui/jobs/StagePage.scala | 149 +++++++++--------- .../org/apache/spark/ui/jobs/StageTable.scala | 4 +- .../spark/ui/storage/BlockManagerUI.scala | 1 + .../apache/spark/ui/storage/IndexPage.scala | 1 - .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../org/apache/spark/util/FileLogger.scala | 12 +- .../scala/org/apache/spark/util/Utils.scala | 4 +- .../ui/jobs/JobProgressListenerSuite.scala | 23 ++- 26 files changed, 189 insertions(+), 187 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 67a28ce7d1ad5..3f93f5862e0eb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,7 +26,6 @@ import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} -import scala.io.Source import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -582,13 +581,15 @@ class SparkContext( /** * Render a previously persisted SparkUI from a set of event logs - * @param dirPath Path of directory containing the event logs + * @param logPath Path of directory containing the event logs */ - def renderPersistedUI(dirPath: String) = { + def renderPersistedUI(logPath: String) = { val oldUI = new SparkUI(this, fromDisk = true) oldUI.start() - val success = oldUI.renderFromDisk(dirPath) - if (success) { + val success = oldUI.renderFromDisk(logPath) + if (!success) { + oldUI.stop() + } else { oldUI.bind() persistedUIs(oldUI.boundPort.get) = oldUI } @@ -610,7 +611,7 @@ class SparkContext( * Stop all persisted UI's rendered in this context */ def stopAllPersistedUIs() = { - persistedUIs.foreach { case (port, _) => stopPersistedUI(port) } + persistedUIs.keys.foreach(stopPersistedUI) } /** Build the union of a list of RDDs. */ diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 18cd5295d15e7..87733d66e51bf 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -79,8 +79,7 @@ private[spark] case object TaskEndReason { (json \ "Class Name").extract[String], (json \ "Description").extract[String], stackTrace, - metrics - ) + metrics) } } 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 847c9931f71f4..24295c5a176e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -541,14 +541,14 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Seq(), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toSeq, properties)) + SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 2b85e84b9992c..2c9057b4af721 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -95,7 +95,7 @@ class JobLogger(val user: String, val logDirName: String) protected def closeLogWriter(jobID: Int) { jobIDToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() - jobIDToStageIDs.get(jobID).foreach(_.foreach{ stageID => + jobIDToStageIDs.get(jobID).foreach(_.foreach { stageID => stageIDToJobID -= stageID }) jobIDToPrintWriter -= jobID @@ -139,19 +139,7 @@ class JobLogger(val user: String, val logDirName: String) } /** - * Log job properties into job log file - * @param jobID ID of the job - * @param properties Properties of the job - */ - protected def recordJobProperties(jobID: Int, properties: Properties) { - if (properties != null) { - val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobID, description, false) - } - } - - /** - * Log task metrics into job log files, including execution info and shuffle metrics + * Record task metrics into job log files, including execution info and shuffle metrics * @param stageID Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info @@ -206,7 +194,7 @@ class JobLogger(val user: String, val logDirName: String) */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo - var taskStatus = taskEnd.taskType + var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) @@ -241,6 +229,18 @@ class JobLogger(val user: String, val logDirName: String) closeLogWriter(jobID) } + /** + * Record job properties into job log file + * @param jobID ID of the job + * @param properties Properties of the job + */ + protected def recordJobProperties(jobID: Int, properties: Properties) { + if (properties != null) { + val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") + jobLogInfo(jobID, description, false) + } + } + /** * When job starts, record job property and stage graph * @param jobStart Job start event @@ -249,8 +249,8 @@ class JobLogger(val user: String, val logDirName: String) val jobID = jobStart.jobId val properties = jobStart.properties createLogWriter(jobID) - buildJobStageDependencies(jobID, jobStart.stageIds) recordJobProperties(jobID, properties) + buildJobStageDependencies(jobID, jobStart.stageIds) jobLogInfo(jobID, "JOB_ID=" + jobID + " STATUS=STARTED") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 8a94a97e675ea..71023a7e194de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,8 +17,9 @@ package org.apache.spark.scheduler -import net.liftweb.json.JsonDSL._ import org.apache.spark.util.Utils + +import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST.JValue import net.liftweb.json.DefaultFormats @@ -32,12 +33,12 @@ private[spark] sealed trait JobResult extends JsonSerializable { private[spark] object JobResult { def fromJson(json: JValue): JobResult = { implicit val format = DefaultFormats - val jobSucceededString = Utils.getFormattedClassName(JobSucceeded) - val jobFailedString = Utils.getFormattedClassName(JobFailed) + val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) + val jobFailed = Utils.getFormattedClassName(JobFailed) (json \ "Result").extract[String] match { - case `jobSucceededString` => JobSucceeded - case `jobFailedString` => jobFailedFromJson(json) + case `jobSucceeded` => JobSucceeded + case `jobFailed` => jobFailedFromJson(json) } } @@ -51,10 +52,10 @@ private[spark] object JobResult { private[spark] case object JobSucceeded extends JobResult +// A failed stage ID of -1 means there is not a particular stage that caused the failure private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult { override def toJson = { val exceptionJson = Utils.exceptionToJson(exception) - super.toJson ~ ("Exception" -> exceptionJson) ~ ("Failed Stage ID" -> failedStageId) 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 78bb61c51fd1d..b1745c67e7a34 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -22,10 +22,10 @@ import java.util.Properties import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.StorageStatus import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ -import org.apache.spark.storage.StorageStatus import net.liftweb.json.DefaultFormats trait JsonSerializable { def toJson: JValue } @@ -103,9 +103,6 @@ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkLi } } -/** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent - /** An event used in the EnvironmentUI */ private[spark] case class SparkListenerLoadEnvironment( jvmInformation: Seq[(String, String)], @@ -113,7 +110,6 @@ private[spark] case class SparkListenerLoadEnvironment( systemProperties: Seq[(String, String)], classpathEntries: Seq[(String, String)]) extends SparkListenerEvent { - override def toJson = { val jvmInformationJson = Utils.mapToJson(jvmInformation.toMap) val sparkPropertiesJson = Utils.mapToJson(sparkProperties.toMap) @@ -137,6 +133,9 @@ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq } } +/** An event used in the listener to shutdown the listener daemon thread. */ +private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent + object SparkListenerEvent { /** * Deserialize a SparkListenerEvent from JSON @@ -150,9 +149,9 @@ object SparkListenerEvent { val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) - val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) val loadEnvironment = Utils.getFormattedClassName(SparkListenerLoadEnvironment) val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) + val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -162,9 +161,9 @@ object SparkListenerEvent { case `taskEnd` => taskEndFromJson(json) case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) - case `shutdown` => SparkListenerShutdown case `loadEnvironment` => loadEnvironmentFromJson(json) case `storageStatusFetch` => storageStatusFetchFromJson(json) + case `shutdown` => SparkListenerShutdown } } @@ -205,8 +204,7 @@ object SparkListenerEvent { new SparkListenerJobStart( (json \ "Job ID").extract[Int], stageIds, - Utils.propertiesFromJson(json \ "Properties") - ) + Utils.propertiesFromJson(json \ "Properties")) } private def jobEndFromJson(json: JValue) = { @@ -308,14 +306,14 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) // Runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => + val runtimePcts = stageCompleted.stageInfo.taskInfos.map { case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", - Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") showDistribution("fetch wait time pct: ", - Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") - showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") + Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") } } @@ -324,7 +322,7 @@ private[spark] object StatsReportListener extends Logging { // For profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) - val probabilities = percentiles.map{_ / 100.0} + val probabilities = percentiles.map(_ / 100.0) val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" def extractDoubleDistribution(stage: SparkListenerStageCompleted, @@ -338,7 +336,7 @@ private[spark] object StatsReportListener extends Logging { def extractLongDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) : Option[Distribution] = { - extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) + extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map(_.toDouble)) } def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { @@ -417,8 +415,8 @@ private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Doubl private object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} - val fetch = fetchTime.map{_ / denom} + val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) + val fetch = fetchTime.map(_ / denom) val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 79ea2b05d0ba3..871c399300056 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -42,6 +42,7 @@ private[spark] class SparkListenerBus extends Logging { val event = eventQueue.take val shutdown = postToListeners(event, sparkListeners) if (shutdown) { + // Get out of the while loop and shutdown the daemon thread return } } @@ -69,7 +70,6 @@ private[spark] class SparkListenerBus extends Logging { case taskEnd: SparkListenerTaskEnd => listeners.foreach(_.onTaskEnd(taskEnd)) case SparkListenerShutdown => - // Get out of the while loop and shutdown the daemon thread return true case _ => } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 621e8a5c81ed3..228ab6162efee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -39,9 +39,7 @@ class StageInfo( mutable.Buffer[(TaskInfo, TaskMetrics)]() ) extends JsonSerializable { - /** - * When this stage was submitted from the DAGScheduler to a TaskScheduler. - */ + /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None var emittedTaskSizeWarning = false @@ -83,7 +81,7 @@ object StageInfo { (TaskInfo.fromJson(info), TaskMetrics.fromJson(metrics)) }.toBuffer - val metrics = new StageInfo( + val stageInfo = new StageInfo( (json \ "Stage ID").extract[Int], (json \ "Stage Name").extract[String], (json \ "RDD Name").extract[String], @@ -91,17 +89,17 @@ object StageInfo { (json \ "Number of Tasks").extract[Int], taskInfo) - metrics.submissionTime = + stageInfo.submissionTime = json \ "Submission Time" match { case JNothing => None case value: JValue => Some(value.extract[Long]) } - metrics.completionTime = + stageInfo.completionTime = json \ "Completion Time" match { case JNothing => None case value: JValue => Some(value.extract[Long]) } - metrics.emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - metrics + stageInfo.emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] + stageInfo } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 76fb57352944f..16768bab62883 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.util.UUID + import org.apache.spark.scheduler.JsonSerializable import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 59c18440da3d8..51d084daee7da 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,8 +19,9 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} -import net.liftweb.json.JsonDSL._ import org.apache.spark.scheduler.JsonSerializable + +import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST.JValue import net.liftweb.json.DefaultFormats diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 54be19c615ac8..da13330727b4b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.storage import org.apache.spark.SparkContext -import BlockManagerMasterActor.BlockStatus import org.apache.spark.util.Utils import org.apache.spark.scheduler.JsonSerializable @@ -26,6 +25,8 @@ import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ import net.liftweb.json.DefaultFormats +import BlockManagerMasterActor.BlockStatus + private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, blocks: Map[BlockId, BlockStatus]) extends JsonSerializable { @@ -71,8 +72,7 @@ case object StorageStatus { new StorageStatus( BlockManagerId.fromJson(json \ "Block Manager ID"), (json \ "Maximum Memory").extract[Long], - blocks - ) + blocks) } } 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 c74f8b4153c13..8b6f9540c5420 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -46,10 +46,12 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend } else { // While each context has only one live SparkUI, it can have many persisted ones // For persisted UI's, climb upwards from the configured / default port - val p = SparkUI.lastPersistedPort.map(_ + 1) - .getOrElse(sc.conf.get("spark.ui.persisted.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt) - SparkUI.lastPersistedPort = Some(p) - p + val nextPort = SparkUI.lastPersistedUIPort match { + case Some(p) => p + 1 + case None => sc.conf.get("spark.ui.persisted.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt + } + SparkUI.lastPersistedUIPort = Some(nextPort) + nextPort } var boundPort: Option[Int] = None var server: Option[Server] = None @@ -104,7 +106,7 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend } /** - * Reconstruct a SparkUI previously persisted from disk from the given path. + * Reconstruct a previously persisted SparkUI from logs residing in the given directory. * Return true if all required log files are found. */ def renderFromDisk(dirPath: String): Boolean = { @@ -164,12 +166,14 @@ private[spark] object SparkUI { val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" // Keep track of the port of the last persisted UI - var lastPersistedPort: Option[Int] = None + var lastPersistedUIPort: Option[Int] = None } /** A SparkListener for logging events, one file per job */ private[spark] class UISparkListener(val name: String, fromDisk: Boolean = false) extends SparkListener with Logging { + + // Log events only if the corresponding UI is not rendered from disk protected val logger: Option[FileLogger] = if (!fromDisk) { Some(new FileLogger(name)) } else { @@ -177,11 +181,7 @@ private[spark] class UISparkListener(val name: String, fromDisk: Boolean = false } protected def logEvent(event: SparkListenerEvent) = { - // Log events only if the corresponding UI is not rendered from disk - if (!fromDisk) { - logWarning("Logging %s".format(Utils.getFormattedClassName(event))) - logger.foreach(_.logLine(compactRender(event.toJson))) - } + logger.foreach(_.logLine(compactRender(event.toJson))) } override def onJobStart(jobStart: SparkListenerJobStart) = logger.foreach(_.start()) @@ -190,7 +190,7 @@ private[spark] class UISparkListener(val name: String, fromDisk: Boolean = false } /** - * A SparkListener that fetches storage information from SparkEnv and logs it as an event. + * A SparkListener that fetches storage information from SparkEnv and logs the corresponding event. * * The frequency at which this occurs is by default every time a stage event is triggered. * This needs not necessarily be the case; a stage can be arbitrarily long, so any failure diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 620af21b7bd70..3410f584ba051 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -38,6 +38,7 @@ private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) def start() { _listener = Some(new EnvironmentListener(sc, fromDisk)) if (!fromDisk) { + // Register for callbacks from this context only if this UI is live sc.addSparkListener(listener) } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 7eb75e8bf93ca..ff5ef6242b237 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -39,6 +39,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false def start() { _listener = Some(new ExecutorsListener(sc, fromDisk)) if (!fromDisk) { + // Register for callbacks from this context only if this UI is live sc.addSparkListener(listener) } } @@ -155,7 +156,6 @@ private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false ) ++ Seq(maxMem) val execValuesString = execValues.map(_.toString) - execFields.zip(execValuesString).toMap } } @@ -195,7 +195,6 @@ private[spark] class ExecutorsListener(sc: SparkContext, fromDisk: Boolean = fal executorToShuffleRead(eid) = executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead } - taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => executorToShuffleWrite(eid) = executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index df1e98013b301..deaeb8a5e0dae 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -17,11 +17,10 @@ package org.apache.spark.ui.jobs +import scala.collection.mutable import scala.xml.Node -import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.util.Utils -import scala.collection.mutable /** Page showing executor summary */ private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { @@ -55,7 +54,7 @@ private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { } private def createExecutorTable() : Seq[Node] = { - // make a executor-id -> address map + // Make an executor-id -> address map val executorIdToAddress = mutable.HashMap[String, String]() val storageStatusList = parent.sc.getExecutorStorageStatus for (statusId <- 0 until storageStatusList.size) { @@ -67,9 +66,8 @@ private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) executorIdToSummary match { - case Some(x) => { - x.toSeq.sortBy(_._1).map{ - case (k,v) => { + case Some(x) => + x.toSeq.sortBy(_._1).map { case (k, v) => { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} @@ -84,8 +82,7 @@ private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { } } - } - case _ => { Seq[Node]() } + case _ => Seq[Node]() } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 21aee456fb122..4fe833c23f6c5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{ListBuffer, HashMap, HashSet} +import scala.collection.mutable.{ListBuffer, HashMap} import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.executor.TaskMetrics @@ -65,7 +65,7 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo val stageId = stage.stageId - // Remove by stageId, rather than by StageInfo, in case the StageInfo is persisted + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from disk poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) completedStages += stage @@ -165,7 +165,7 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f } val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is persisted + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from disk tasksActive.remove(taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = @@ -214,7 +214,7 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f jobEnd.jobResult match { case JobFailed(_, stageId) => activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is persisted + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from disk activeStages.remove(s.stageId) poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) failedStages += s @@ -230,4 +230,4 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f private[spark] case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, - exception: Option[ExceptionFailure] = None) \ No newline at end of file + exception: Option[ExceptionFailure] = None) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index d4e01bbc9d0b3..4ed36937dfe44 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -23,8 +23,8 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.SparkContext +import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ @@ -32,7 +32,7 @@ private[spark] class JobProgressUI(val sc: SparkContext, fromDisk: Boolean = fal private var _listener: Option[JobProgressListener] = None private val indexPage = new IndexPage(this, fromDisk) private val stagePage = new StagePage(this, fromDisk) - private val poolPage = new PoolPage(this) + private val poolPage = new PoolPage(this, fromDisk) val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") @@ -41,6 +41,7 @@ private[spark] class JobProgressUI(val sc: SparkContext, fromDisk: Boolean = fal def start() { _listener = Some(new JobProgressListener(sc, fromDisk)) if (!fromDisk) { + // Register for callbacks from this context only if this UI is live sc.addSparkListener(listener) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 2329bba779462..c9897d4fa4ca9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -18,7 +18,6 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0d876999f2758..f42d94f3fe7ab 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,12 +23,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ExceptionFailure -import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.TaskInfo /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) { @@ -214,81 +211,85 @@ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) - taskData match { case TaskUIData(info, metrics, exception) => - val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) - else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") - val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) - val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) - - val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") - - val maybeShuffleWrite = - metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") - - val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) - val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) - }.getOrElse("") - - val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) - val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") - val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") - - val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled) - val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") - val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") - - - {info.index} - {info.taskId} - {info.status} - {info.taskLocality} - {info.host} - {dateFmt.format(new Date(info.launchTime))} - - {formatDuration} + val info = taskData.taskInfo + val metrics = taskData.taskMetrics + val exception = taskData.exception + + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(_.executorRunTime).getOrElse(1L) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + + val maybeShuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + + val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else parent.formatDuration(ms) + }.getOrElse("") + + val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) + val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") + val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") + + val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled) + val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") + val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") + + + {info.index} + {info.taskId} + {info.status} + {info.taskLocality} + {info.host} + {dateFmt.format(new Date(info.launchTime))} + + {formatDuration} + + + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + + + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + + {if (shuffleRead) { + + {shuffleReadReadable} + + }} + {if (shuffleWrite) { + + {writeTimeReadable} + + + {shuffleWriteReadable} + + }} + {if (bytesSpilled) { + + {memoryBytesSpilledReadable} - - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + + {diskBytesSpilledReadable} - - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - - {if (shuffleRead) { - - {shuffleReadReadable} - - }} - {if (shuffleWrite) { - - {writeTimeReadable} - - - {shuffleWriteReadable} - - }} - {if (bytesSpilled) { - - {memoryBytesSpilledReadable} - - - {diskBytesSpilledReadable} - - }} - {exception.map(e => + }} + + {exception.map { e => {e.className} ({e.description})
{fmtStackTrace(e.stackTrace)} -
).getOrElse("")} - - - } + + }.getOrElse("")} + + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 0831ea7084a69..5b4027f90172a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -39,7 +39,7 @@ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { } } - /** Render a special table that merges two header cells. */ + /** Special table that merges two header cells. */ private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -80,7 +80,7 @@ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
{d}
{nameLink}
).getOrElse(nameLink) val submissionTime = s.submissionTime match { - case Some(d) => dateFmt.format(new Date(d)) + case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index bbd75164c1516..dc380c98f30e0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -36,6 +36,7 @@ private[spark] class BlockManagerUI(val sc: SparkContext, fromDisk: Boolean = fa def start() { _listener = Some(new BlockManagerListener(sc, fromDisk)) if (!fromDisk) { + // Register for callbacks from this context only if this UI is live sc.addSparkListener(listener) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 19e173cf758da..579d11c67064f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -26,7 +26,6 @@ import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.Utils - /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { private val sc = parent.sc diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 6107595d2de62..de7efa1bee543 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -50,7 +50,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) val blocks = blockStatuses.map { - case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("Unknown"))) } val blockTable = listingTable(blockHeader, blockRow, blocks) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f25a33c73989b..f0fbe34028773 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import java.io._ import java.text.SimpleDateFormat import java.util.Date + import org.apache.spark.Logging /** @@ -44,7 +45,7 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU private var writer: Option[PrintWriter] = { createLogDir() - Some(createWriter()) // Overwrite any existing file + Some(createWriter()) } def this() = this(System.getProperty("user.name", ""), @@ -66,6 +67,7 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU /** Create a new writer to the file identified with the given path */ private def createWriter() = { + // Overwrite any existing file val fileWriter = new FileWriter(logDir + fileIndex) val bufferedWriter = new BufferedWriter(fileWriter) new PrintWriter(bufferedWriter) @@ -108,11 +110,9 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU /** Start a new writer (for a new file) if there does not already exist one */ def start() = { - writer match { - case Some(w) => - case None => - fileIndex += 1 - writer = Some(createWriter()) + writer.getOrElse { + fileIndex += 1 + Some(createWriter()) } } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9bccc8a8b64ed..c8df917c01558 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -900,9 +900,7 @@ private[spark] object Utils extends Logging { def propertiesFromJson(json: JValue): Properties = { val properties = new Properties() if (json != JNothing) { - mapFromJson(json).map { case (k, v) => - properties.setProperty(k, v) - } + mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } } properties } 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 524c80d1520e6..9483fd8fb10f2 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 @@ -18,9 +18,11 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite + import org.apache.spark.scheduler._ import org.apache.spark.{LocalSparkContext, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { @@ -37,16 +39,19 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + var task = new ShuffleMapTask(0, null, null, 0, null) + val taskType = Utils.getFormattedClassName(task) + listener.onTaskEnd( + new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd( + new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.size == 1) // finish this task, should get updated duration @@ -54,8 +59,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd( + new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 2000) @@ -64,8 +70,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd( + new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) .shuffleRead == 1000) } From 904c7294ac221a0cd9806af843219aaa8a847085 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Feb 2014 22:06:46 -0800 Subject: [PATCH 21/68] Fix another major bug Previously, rendering the old, persisted UI continues to trigger load environment and storage status fetch events. These are now only triggered for the live UI. A related TODO: Under JobProgressUI, the total duration is inaccurate; right now it uses the time when the old UI is revived, rather than when it was live. This should be fixed. --- .../org/apache/spark/scheduler/SparkListenerBus.scala | 4 ++++ core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 6 +++++- .../main/scala/org/apache/spark/ui/env/EnvironmentUI.scala | 4 +++- .../main/scala/org/apache/spark/ui/jobs/IndexPage.scala | 4 ++-- .../src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala | 7 +++---- .../scala/org/apache/spark/ui/storage/BlockManagerUI.scala | 4 ++-- .../main/scala/org/apache/spark/ui/storage/IndexPage.scala | 6 ++++-- .../main/scala/org/apache/spark/ui/storage/RDDPage.scala | 6 ++++-- 8 files changed, 27 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 871c399300056..fa3c3d03e2616 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -69,6 +69,10 @@ private[spark] class SparkListenerBus extends Logging { listeners.foreach(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => listeners.foreach(_.onTaskEnd(taskEnd)) + case loadEnvironment: SparkListenerLoadEnvironment => + listeners.foreach(_.onLoadEnvironment(loadEnvironment)) + case storageStatusFetch: SparkListenerStorageStatusFetch => + listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) case SparkListenerShutdown => return true case _ => 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 8b6f9540c5420..0e9aa1c4835bc 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -119,11 +119,15 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend return false } listeners.map { listener => - val path = "%s/%s/".format(dirPath.stripSuffix("/"), listener.name) + val name = listener.name + val path = "%s/%s/".format(dirPath.stripSuffix("/"), name) val dir = new File(path) if (dir.exists && dir.isDirectory) { val files = dir.listFiles Option(files).foreach { files => files.foreach(processPersistedEventLog(_, listener)) } + if (files.size == 0) { + logWarning("No logs found for %s; %s is empty".format(name, path)) + } } else { logWarning("%s not found when rendering persisted Spark Web UI!".format(path)) success = false diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 3410f584ba051..4b33b3b9678f2 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -48,7 +48,9 @@ private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) ) def render(request: HttpServletRequest): Seq[Node] = { - listener.loadEnvironment() + if (!fromDisk) { + listener.loadEnvironment() + } val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 77fb8c1105616..30d78e5be92da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -38,8 +38,8 @@ private[spark] class IndexPage(parent: JobProgressUI, fromDisk: Boolean = false) val now = System.currentTimeMillis() val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, - parent) + val completedStagesTable = + new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) val pools = sc.getAllPools diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index bd78273c8dc01..a286988c89003 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -19,15 +19,14 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} -import scala.collection.mutable.HashSet +import scala.xml.Node -import org.apache.spark.scheduler.Stage import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.Logging /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI, fromDisk: Boolean = false) { +private[spark] class PoolPage(parent: JobProgressUI, fromDisk: Boolean = false) extends Logging { private val sc = parent.sc private def listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index dc380c98f30e0..a56ff6a75986c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -28,8 +28,8 @@ import org.apache.spark.ui.StorageStatusFetchSparkListener /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] class BlockManagerUI(val sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[BlockManagerListener] = None - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) + private val indexPage = new IndexPage(this, fromDisk) + private val rddPage = new RDDPage(this, fromDisk) def listener = _listener.get diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 579d11c67064f..c735e32c10a0f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -27,12 +27,14 @@ import org.apache.spark.ui.Page._ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[spark] class IndexPage(parent: BlockManagerUI) { +private[spark] class IndexPage(parent: BlockManagerUI, fromDisk: Boolean = false) { private val sc = parent.sc private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() + if (!fromDisk) { + listener.fetchStorageStatus() + } val storageStatusList = listener.storageStatusList // Calculate macro-level statistics diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index de7efa1bee543..e3d7bba1abfdb 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -29,12 +29,14 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[spark] class RDDPage(parent: BlockManagerUI) { +private[spark] class RDDPage(parent: BlockManagerUI, fromDisk: Boolean = false) { private val sc = parent.sc private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() + if (!fromDisk) { + listener.fetchStorageStatus() + } val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt val filteredStorageStatusList = From 427301371117e9e7889f5df0f6bba51e5916e425 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 18 Feb 2014 15:27:39 -0800 Subject: [PATCH 22/68] Add a gateway SparkListener to simplify event logging Instead of having each SparkListener log an independent set of events, centralize event logging to avoid differentiating events across UI's and thus duplicating logged events. Also rename the "fromDisk" parameter to "live". TODO: Storage page currently still relies on the previous SparkContext and is not rendering correctly. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 142 +++++----------- .../org/apache/spark/ui/UISparkListener.scala | 152 ++++++++++++++++++ .../apache/spark/ui/env/EnvironmentUI.scala | 93 ++++++----- .../apache/spark/ui/exec/ExecutorsUI.scala | 31 ++-- .../org/apache/spark/ui/jobs/IndexPage.scala | 6 +- .../spark/ui/jobs/JobProgressListener.scala | 14 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 21 ++- .../org/apache/spark/ui/jobs/PoolPage.scala | 7 +- .../org/apache/spark/ui/jobs/StagePage.scala | 14 +- .../spark/ui/storage/BlockManagerUI.scala | 26 +-- .../apache/spark/ui/storage/IndexPage.scala | 14 +- .../org/apache/spark/ui/storage/RDDPage.scala | 18 +-- .../org/apache/spark/util/FileLogger.scala | 2 +- 15 files changed, 315 insertions(+), 229 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/UISparkListener.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f93f5862e0eb..f385aac3fffc0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -584,7 +584,7 @@ class SparkContext( * @param logPath Path of directory containing the event logs */ def renderPersistedUI(logPath: String) = { - val oldUI = new SparkUI(this, fromDisk = true) + val oldUI = new SparkUI(this, live = false) oldUI.start() val success = oldUI.renderFromDisk(logPath) if (!success) { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index da13330727b4b..af32b2eafe568 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -109,7 +109,7 @@ object StorageUtils { blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap } - /* Given a list of BlockStatus objets, returns information for each RDD */ + /* Given a list of BlockStatus objects, returns information for each RDD */ def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { 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 0e9aa1c4835bc..af1fc32653617 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -24,24 +24,22 @@ import scala.io.Source import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.{Logging, SparkContext, SparkEnv} +import org.apache.spark.scheduler._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{FileLogger, Utils} -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatus +import org.apache.spark.util.Utils import net.liftweb.json._ -import net.liftweb.json.JsonAST.compactRender import it.unimi.dsi.fastutil.io.FastBufferedInputStream /** Top level user interface for Spark */ -private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extends Logging { +private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = if (!fromDisk) { + val port = if (live) { sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt } else { // While each context has only one live SparkUI, it can have many persisted ones @@ -60,10 +58,10 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("/", createRedirectHandler("/stages")) ) - private val storage = new BlockManagerUI(sc, fromDisk) - private val jobs = new JobProgressUI(sc, fromDisk) - private val env = new EnvironmentUI(sc, fromDisk) - private val exec = new ExecutorsUI(sc, fromDisk) + private val storage = new BlockManagerUI(this, live) + private val jobs = new JobProgressUI(this, live) + private val env = new EnvironmentUI(this, live) + private val exec = new ExecutorsUI(this, live) // Add MetricsServlet handlers by default private val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers @@ -71,8 +69,14 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers - // Listeners are not ready until SparkUI has started - private def listeners = Seq(storage.listener, jobs.listener, env.listener, exec.listener) + // Maintain a gateway listener for all events to simplify event logging + private var _gatewayListener: Option[GatewayUISparkListener] = None + + def gatewayListener = _gatewayListener.getOrElse { + val gateway = new GatewayUISparkListener(live) + _gatewayListener = Some(gateway) + gateway + } /** Bind the HTTP server which backs this web interface */ def bind() { @@ -94,6 +98,11 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend // DAGScheduler() requires that the port of this server is known // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + + if (live) { + // Listen for new events only if this UI is live + sc.addSparkListener(gatewayListener) + } storage.start() jobs.start() env.start() @@ -107,40 +116,35 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend /** * Reconstruct a previously persisted SparkUI from logs residing in the given directory. - * Return true if all required log files are found. + * Return true if log files are found and processed. */ def renderFromDisk(dirPath: String): Boolean = { - var success = true - if (fromDisk) { - val logDir = new File(dirPath) - if (!logDir.exists || !logDir.isDirectory) { - logWarning("Given invalid directory %s when rendering persisted Spark Web UI!" - .format(dirPath)) - return false - } - listeners.map { listener => - val name = listener.name - val path = "%s/%s/".format(dirPath.stripSuffix("/"), name) - val dir = new File(path) - if (dir.exists && dir.isDirectory) { - val files = dir.listFiles - Option(files).foreach { files => files.foreach(processPersistedEventLog(_, listener)) } - if (files.size == 0) { - logWarning("No logs found for %s; %s is empty".format(name, path)) - } - } else { - logWarning("%s not found when rendering persisted Spark Web UI!".format(path)) - success = false - } - } + // Live UI's should never invoke this + assert(!live) + + // Check validity of the given path + val logDir = new File(dirPath) + if (!logDir.exists || !logDir.isDirectory) { + logWarning("Given invalid log path %s when rendering persisted Spark Web UI!" + .format(dirPath)) + return false } - success + val logFiles = logDir.listFiles.filter(_.isFile) + if (logFiles.size == 0) { + logWarning("No logs found in given directory %s when rendering persisted Spark Web UI!" + .format(dirPath)) + return false + } + + // Replay events in each event log + logFiles.foreach(processEventLog) + true } /** - * Register each event logged in the given file with the corresponding listener in order + * Replay each event in the order maintained in the given log to the gateway listener */ - private def processPersistedEventLog(file: File, listener: SparkListener) = { + private def processEventLog(file: File) = { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) var currentLine = "" @@ -149,7 +153,7 @@ private[spark] class SparkUI(sc: SparkContext, fromDisk: Boolean = false) extend lines.foreach { line => currentLine = line val event = SparkListenerEvent.fromJson(parse(line)) - sc.dagScheduler.listenerBus.postToListeners(event, Seq(listener)) + sc.dagScheduler.listenerBus.postToListeners(event, Seq(gatewayListener)) } } catch { case e: Exception => @@ -172,61 +176,3 @@ private[spark] object SparkUI { // Keep track of the port of the last persisted UI var lastPersistedUIPort: Option[Int] = None } - -/** A SparkListener for logging events, one file per job */ -private[spark] class UISparkListener(val name: String, fromDisk: Boolean = false) - extends SparkListener with Logging { - - // Log events only if the corresponding UI is not rendered from disk - protected val logger: Option[FileLogger] = if (!fromDisk) { - Some(new FileLogger(name)) - } else { - None - } - - protected def logEvent(event: SparkListenerEvent) = { - logger.foreach(_.logLine(compactRender(event.toJson))) - } - - override def onJobStart(jobStart: SparkListenerJobStart) = logger.foreach(_.start()) - - override def onJobEnd(jobEnd: SparkListenerJobEnd) = logger.foreach(_.close()) -} - -/** - * A SparkListener that fetches storage information from SparkEnv and logs the corresponding event. - * - * The frequency at which this occurs is by default every time a stage event is triggered. - * This needs not necessarily be the case; a stage can be arbitrarily long, so any failure - * in the middle of a stage causes the storage status for that stage to be lost. - */ -private[spark] class StorageStatusFetchSparkListener( - name: String, - sc: SparkContext, - fromDisk: Boolean = false) - extends UISparkListener(name, fromDisk) { - var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus - - /** - * Fetch storage information from SparkEnv, which involves a query to the driver. This is - * expensive and should be invoked sparingly. - */ - def fetchStorageStatus() { - val storageStatus = sc.getExecutorStorageStatus - val event = new SparkListenerStorageStatusFetch(storageStatus) - onStorageStatusFetch(event) - } - - /** - * Update local state with fetch result, and log the appropriate event - */ - override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { - storageStatusList = storageStatusFetch.storageStatusList - logEvent(storageStatusFetch) - logger.foreach(_.flush()) - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = fetchStorageStatus() - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = fetchStorageStatus() -} diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala new file mode 100644 index 0000000000000..3d234a4defbdf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -0,0 +1,152 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.SparkListenerLoadEnvironment +import org.apache.spark.scheduler.SparkListenerJobStart +import org.apache.spark.SparkContext +import org.apache.spark.storage.StorageStatus +import org.apache.spark.util.FileLogger + +import net.liftweb.json.JsonAST._ + +private[spark] case class UISparkListener(gateway: GatewayUISparkListener) extends SparkListener { + // Register with gateway listener + gateway.registerSparkListener(this) +} + +/** + * A SparkListener that serves as a gateway for all events posted to the UI. + * + * GatewayUISparkListener achieves two functions: + * + * (1) If the UI is live, GatewayUISparkListener posts each event to all attached listeners + * then logs it as JSON. This centralizes event logging and avoids having all attached + * listeners log the events on their own. By default, GatewayUISparkListener logs one + * file per job, though this needs not be the case. + * + * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized + * from the event logs to all attached listeners. + */ +private[spark] class GatewayUISparkListener(live: Boolean) extends SparkListener { + + // Log events only if the UI is live + private val logger: Option[FileLogger] = if (live) Some(new FileLogger()) else None + + // Children listeners for which this gateway is responsible + private val listeners = ArrayBuffer[UISparkListener]() + + def registerSparkListener(listener: UISparkListener) = { + listeners += listener + } + + /** Log the event as JSON */ + private def logEvent(event: SparkListenerEvent) { + logger.foreach(_.logLine(compactRender(event.toJson))) + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + listeners.foreach(_.onStageSubmitted(stageSubmitted)) + logEvent(stageSubmitted) + logger.foreach(_.flush()) + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + listeners.foreach(_.onStageCompleted(stageCompleted)) + logEvent(stageCompleted) + logger.foreach(_.flush()) + } + + override def onTaskStart(taskStart: SparkListenerTaskStart) { + listeners.foreach(_.onTaskStart(taskStart)) + logEvent(taskStart) + } + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { + listeners.foreach(_.onTaskGettingResult(taskGettingResult)) + logEvent(taskGettingResult) + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + listeners.foreach(_.onTaskEnd(taskEnd)) + logEvent(taskEnd) + } + + override def onJobStart(jobStart: SparkListenerJobStart) { + listeners.foreach(_.onJobStart(jobStart)) + logger.foreach(_.start()) + logEvent(jobStart) + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + listeners.foreach(_.onJobEnd(jobEnd)) + logEvent(jobEnd) + logger.foreach(_.close()) + } + + override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { + listeners.foreach(_.onLoadEnvironment(loadEnvironment)) + logEvent(loadEnvironment) + logger.foreach(_.flush()) + } + + override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { + listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) + logEvent(storageStatusFetch) + logger.foreach(_.flush()) + } +} + +/** + * A SparkListener that fetches storage information from SparkEnv. + * + * The frequency at which this occurs is by default every time a stage event is triggered. + * This needs not be the case, however; a stage can be arbitrarily long, so any failure + * in the middle of a stage causes the storage status for that stage to be lost. + */ +private[spark] class StorageStatusFetchSparkListener( + sc: SparkContext, + gateway: GatewayUISparkListener, + live: Boolean) + extends UISparkListener(gateway) { + var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus + + /** + * Fetch storage information from SparkEnv, which involves a query to the driver. This is + * expensive and should be invoked sparingly. + */ + def fetchStorageStatus() { + if (live) { + // Fetch only this is a live UI + val storageStatus = sc.getExecutorStorageStatus + val event = new SparkListenerStorageStatusFetch(storageStatus) + gateway.onStorageStatusFetch(event) + } + } + + /** + * Update local state with fetch result, and log the appropriate event + */ + override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { + storageStatusList = storageStatusFetch.storageStatusList + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = fetchStorageStatus() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = fetchStorageStatus() +} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 4b33b3b9678f2..924f2c03356e8 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -28,19 +28,18 @@ import org.eclipse.jetty.server.Handler import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.{UISparkListener, UIUtils} import org.apache.spark.ui.Page.Environment +import org.apache.spark.ui._ + +private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) { + val sc = parent.sc -private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) { private var _listener: Option[EnvironmentListener] = None + def listener = _listener.get def start() { - _listener = Some(new EnvironmentListener(sc, fromDisk)) - if (!fromDisk) { - // Register for callbacks from this context only if this UI is live - sc.addSparkListener(listener) - } + _listener = Some(new EnvironmentListener(sc, parent.gatewayListener, live)) } def getHandlers = Seq[(String, Handler)]( @@ -48,9 +47,7 @@ private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) ) def render(request: HttpServletRequest): Seq[Node] = { - if (!fromDisk) { - listener.loadEnvironment() - } + listener.loadEnvironment() val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( @@ -78,56 +75,58 @@ private[spark] class EnvironmentUI(sc: SparkContext, fromDisk: Boolean = false) } /** - * A SparkListener that prepares and logs information to be displayed on the Environment UI + * A SparkListener that prepares information to be displayed on the EnvironmentUI */ -private[spark] class EnvironmentListener(sc: SparkContext, fromDisk: Boolean = false) - extends UISparkListener("environment-ui", fromDisk) { +private[spark] class EnvironmentListener( + sc: SparkContext, + gateway: GatewayUISparkListener, + live: Boolean) + extends UISparkListener(gateway) { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() /** Gather JVM, spark, system and classpath properties */ - def loadEnvironment() = { - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - val sparkProperties = sc.conf.getAll.sorted - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted - - // Trigger SparkListenerLoadEnvironment - val loadEnvironment = new SparkListenerLoadEnvironment( - jvmInformation, sparkProperties, otherProperties, classPaths) - onLoadEnvironment(loadEnvironment) + def loadEnvironment() { + if (live) { + // Load environment only this is a live UI + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + val sparkProperties = sc.conf.getAll.sorted + val systemProperties = System.getProperties.iterator.toSeq + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + val classPathEntries = classPathProperty._2 + .split(sc.conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted + + // Trigger SparkListenerLoadEnvironment + val loadEnvironment = new SparkListenerLoadEnvironment( + jvmInformation, sparkProperties, otherProperties, classPaths) + gateway.onLoadEnvironment(loadEnvironment) + } } - /** Prepare environment information for UI to render, and log the corresponding event */ - override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) = { + /** Prepare environment information for UI to render */ + override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { jvmInformation = loadEnvironment.jvmInformation sparkProperties = loadEnvironment.sparkProperties systemProperties = loadEnvironment.systemProperties classpathEntries = loadEnvironment.classpathEntries - logEvent(loadEnvironment) - logger.foreach(_.flush()) } - override def onJobStart(jobStart: SparkListenerJobStart) = { - loadEnvironment() - } + override def onJobStart(jobStart: SparkListenerJobStart) = loadEnvironment() } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index ff5ef6242b237..9dfcb50d41849 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -25,23 +25,21 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler import org.apache.spark.{SparkContext, ExceptionFailure} +import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{StorageStatusFetchSparkListener, UIUtils} +import org.apache.spark.ui._ import org.apache.spark.util.Utils -import org.apache.spark.scheduler.SparkListenerTaskEnd -import org.apache.spark.scheduler.SparkListenerTaskStart -private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false) { +private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) { + val sc = parent.sc + private var _listener: Option[ExecutorsListener] = None + def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener(sc, fromDisk)) - if (!fromDisk) { - // Register for callbacks from this context only if this UI is live - sc.addSparkListener(listener) - } + _listener = Some(new ExecutorsListener(sc, parent.gatewayListener, live)) } def getHandlers = Seq[(String, Handler)]( @@ -49,9 +47,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false ) def render(request: HttpServletRequest): Seq[Node] = { - if (!fromDisk) { - listener.fetchStorageStatus() - } + listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) @@ -161,10 +157,13 @@ private[spark] class ExecutorsUI(val sc: SparkContext, fromDisk: Boolean = false } /** - * A SparkListener that prepares and logs information to be displayed on the Executors UI + * A SparkListener that prepares information to be displayed on the ExecutorsUI */ -private[spark] class ExecutorsListener(sc: SparkContext, fromDisk: Boolean = false) - extends StorageStatusFetchSparkListener("executors-ui", sc, fromDisk) { +private[spark] class ExecutorsListener( + sc: SparkContext, + gateway: GatewayUISparkListener, + live: Boolean) + extends StorageStatusFetchSparkListener(sc, gateway, live) { val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() @@ -175,7 +174,6 @@ private[spark] class ExecutorsListener(sc: SparkContext, fromDisk: Boolean = fal override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = formatExecutorId(taskStart.taskInfo.executorId) executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - logEvent(taskStart) } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -200,7 +198,6 @@ private[spark] class ExecutorsListener(sc: SparkContext, fromDisk: Boolean = fal executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten } } - logEvent(taskEnd) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 30d78e5be92da..abc1114e01162 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -23,10 +23,10 @@ import scala.xml.{NodeSeq, Node} import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ -private[spark] class IndexPage(parent: JobProgressUI, fromDisk: Boolean = false) { +private[spark] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private def listener = parent.listener @@ -80,7 +80,7 @@ private[spark] class IndexPage(parent: JobProgressUI, fromDisk: Boolean = false)

Failed Stages ({failedStages.size})

++ failedStagesTable.toNodeSeq - headerSparkPage(content, sc, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, sc, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 4fe833c23f6c5..1cfac6de8c826 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.{ListBuffer, HashMap} import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.ui.UISparkListener +import org.apache.spark.ui.{GatewayUISparkListener, UISparkListener} /** * Tracks task-level information to be displayed in the UI. @@ -31,8 +31,10 @@ import org.apache.spark.ui.UISparkListener * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = false) - extends UISparkListener("job-progress-ui", fromDisk) { +private[spark] class JobProgressListener( + sc: SparkContext, + gateway: GatewayUISparkListener) + extends UISparkListener(gateway) { // How many stages to remember val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) @@ -70,7 +72,6 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f activeStages.remove(stageId) completedStages += stage trimIfNecessary(completedStages) - logEvent(stageCompleted) } /** If stages is too large, remove and garbage collect old stages */ @@ -111,7 +112,6 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) stages(stage.stageId) = stage - logEvent(stageSubmitted) } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { @@ -122,7 +122,6 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) stageIdToTaskInfos(sid) = taskMap - logEvent(taskStart) } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) @@ -207,7 +206,6 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f val taskInfo = taskEnd.taskInfo taskMap(taskInfo.taskId) = new TaskUIData(taskInfo, metrics, failureInfo) stageIdToTaskInfos(sid) = taskMap - logEvent(taskEnd) } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { @@ -222,8 +220,6 @@ private[spark] class JobProgressListener(sc: SparkContext, fromDisk: Boolean = f } case _ => } - logEvent(jobEnd) - logger.foreach(_.close()) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 4ed36937dfe44..6f54b099dbb23 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -23,27 +23,24 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(val sc: SparkContext, fromDisk: Boolean = false) { - private var _listener: Option[JobProgressListener] = None - private val indexPage = new IndexPage(this, fromDisk) - private val stagePage = new StagePage(this, fromDisk) - private val poolPage = new PoolPage(this, fromDisk) - +private[spark] class JobProgressUI(parent: SparkUI, live: Boolean) { val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val sc = parent.sc + + private val indexPage = new IndexPage(this) + private val stagePage = new StagePage(this) + private val poolPage = new PoolPage(this) + private var _listener: Option[JobProgressListener] = None def listener = _listener.get def start() { - _listener = Some(new JobProgressListener(sc, fromDisk)) - if (!fromDisk) { - // Register for callbacks from this context only if this UI is live - sc.addSparkListener(listener) - } + _listener = Some(new JobProgressListener(sc, parent.gatewayListener)) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index a286988c89003..cce54fca6e168 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,12 +21,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ -import org.apache.spark.Logging +import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI, fromDisk: Boolean = false) extends Logging { +private[spark] class PoolPage(parent: JobProgressUI) { private val sc = parent.sc private def listener = parent.listener @@ -46,7 +45,7 @@ private[spark] class PoolPage(parent: JobProgressUI, fromDisk: Boolean = false) val content =

Summary

++ poolTable.toNodeSeq ++

{activeStages.size} Active Stages

++ activeStagesTable.toNodeSeq - headerSparkPage(content, sc, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, sc, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index f42d94f3fe7ab..09be0c323f696 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,14 +23,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) { +private[spark] class StagePage(parent: JobProgressUI) { private val sc = parent.sc - private def dateFmt = parent.dateFmt + private val dateFmt = parent.dateFmt private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -43,7 +43,7 @@ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false)

Summary Metrics

No tasks have started yet

Tasks

No tasks have started yet - return headerSparkPage(content, sc, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, sc, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -103,7 +103,7 @@ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ Seq("Errors") - val taskTable = listingTable( + val taskTable = UIUtils.listingTable( taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics @@ -192,7 +192,7 @@ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false) val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") def quantileRow(data: Seq[String]): Seq[Node] =
{data.map(d => )} - Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) + Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } val executorTable = new ExecutorTable(stageId, parent) val content = @@ -202,7 +202,7 @@ private[spark] class StagePage(parent: JobProgressUI, fromDisk: Boolean = false)

Aggregated Metrics by Executor

++ executorTable.toNodeSeq ++

Tasks

++ taskTable - headerSparkPage(content, sc, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, sc, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a56ff6a75986c..2cc4d44fb78c6 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -23,22 +23,20 @@ import org.eclipse.jetty.server.Handler import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.StorageStatusFetchSparkListener +import org.apache.spark.ui.{GatewayUISparkListener, SparkUI, StorageStatusFetchSparkListener} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(val sc: SparkContext, fromDisk: Boolean = false) { +private[spark] class BlockManagerUI(parent: SparkUI, live: Boolean) { + val sc = parent.sc + + private val indexPage = new IndexPage(this) + private val rddPage = new RDDPage(this) private var _listener: Option[BlockManagerListener] = None - private val indexPage = new IndexPage(this, fromDisk) - private val rddPage = new RDDPage(this, fromDisk) def listener = _listener.get def start() { - _listener = Some(new BlockManagerListener(sc, fromDisk)) - if (!fromDisk) { - // Register for callbacks from this context only if this UI is live - sc.addSparkListener(listener) - } + _listener = Some(new BlockManagerListener(sc, parent.gatewayListener, live)) } def getHandlers = Seq[(String, Handler)]( @@ -47,5 +45,11 @@ private[spark] class BlockManagerUI(val sc: SparkContext, fromDisk: Boolean = fa ) } -private[spark] class BlockManagerListener(sc: SparkContext, fromDisk: Boolean = false) - extends StorageStatusFetchSparkListener("block-manager-ui", sc, fromDisk) +/** + * A SparkListener that prepares information to be displayed on the BlockManagerUI + */ +private[spark] class BlockManagerListener( + sc: SparkContext, + gateway: GatewayUISparkListener, + live: Boolean) + extends StorageStatusFetchSparkListener(sc, gateway, live) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index c735e32c10a0f..7ad1b3909c506 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,25 +22,23 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{RDDInfo, StorageUtils} -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[spark] class IndexPage(parent: BlockManagerUI, fromDisk: Boolean = false) { +private[spark] class IndexPage(parent: BlockManagerUI) { private val sc = parent.sc private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - if (!fromDisk) { - listener.fetchStorageStatus() - } + listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList // Calculate macro-level statistics val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val content = listingTable(rddHeader, rddRow, rdds) - headerSparkPage(content, sc, "Storage ", Storage) + val content = UIUtils.listingTable(rddHeader, rddRow, rdds) + UIUtils.headerSparkPage(content, sc, "Storage ", Storage) } /** Header fields for the RDD table */ @@ -56,7 +54,7 @@ private[spark] class IndexPage(parent: BlockManagerUI, fromDisk: Boolean = false private def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index e3d7bba1abfdb..32db30edad08a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -23,20 +23,18 @@ import scala.xml.Node import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils} import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[spark] class RDDPage(parent: BlockManagerUI, fromDisk: Boolean = false) { +private[spark] class RDDPage(parent: BlockManagerUI) { private val sc = parent.sc private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - if (!fromDisk) { - listener.fetchStorageStatus() - } + listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt val filteredStorageStatusList = @@ -45,16 +43,16 @@ private[spark] class RDDPage(parent: BlockManagerUI, fromDisk: Boolean = false) // Worker table val workers = filteredStorageStatusList.map((id, _)) - val workerTable = listingTable(workerHeader, workerRow, workers) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray. sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) - val blocks = blockStatuses.map { - case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("Unknown"))) + val blocks = blockStatuses.map { case (id, status) => + (id, status, blockLocations.get(id).getOrElse(Seq("Unknown"))) } - val blockTable = listingTable(blockHeader, blockRow, blocks) + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) val content =
@@ -98,7 +96,7 @@ private[spark] class RDDPage(parent: BlockManagerUI, fromDisk: Boolean = false)
; - headerSparkPage(content, sc, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, sc, "RDD Storage Info for " + rddInfo.name, Storage) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f0fbe34028773..a483489200700 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -112,7 +112,7 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU def start() = { writer.getOrElse { fileIndex += 1 - Some(createWriter()) + writer = Some(createWriter()) } } } From 64d2ce1efee3aa5a8166c5fe108932b2279217fc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 18 Feb 2014 18:29:21 -0800 Subject: [PATCH 23/68] Fix BlockManagerUI bug by introducing new event Previously, the storage information of persisted RDD's continued to rely on the old SparkContext, which is no longer accessible if the UI is rendered from disk. This fix solves it by introducing an event, SparkListenerGetRDDInfo, which captures this information. Per discussion with Patrick, an alternative is to encapsulate this information within SparkListenerTaskEnd. This would bypass the need to create a new event, but would also require a non-trivial refactor of BlockManager / BlockStore. --- .../spark/scheduler/SparkListener.scala | 45 ++++++++++++++----- .../spark/scheduler/SparkListenerBus.scala | 2 + .../apache/spark/storage/StorageUtils.scala | 41 ++++++++++++++--- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../org/apache/spark/ui/UISparkListener.scala | 9 ++-- .../scala/org/apache/spark/ui/UIUtils.scala | 6 +-- .../apache/spark/ui/env/EnvironmentUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 5 ++- .../spark/ui/storage/BlockManagerUI.scala | 28 +++++++++++- .../apache/spark/ui/storage/IndexPage.scala | 11 +++-- .../org/apache/spark/ui/storage/RDDPage.scala | 6 +-- .../spark/util/TimeStampedHashMap.scala | 1 - 15 files changed, 125 insertions(+), 42 deletions(-) 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 b1745c67e7a34..0ac7462ee6aeb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.storage.StorageStatus +import org.apache.spark.storage.{RDDInfo, StorageStatus} import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ @@ -123,7 +123,7 @@ private[spark] case class SparkListenerLoadEnvironment( } } -/** An event used in the ExecutorUI to fetch storage status from SparkEnv */ +/** An event used in the ExecutorsUI and BlockManagerUI to fetch storage status from SparkEnv */ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq[StorageStatus]) extends SparkListenerEvent { override def toJson = { @@ -133,6 +133,16 @@ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq } } +/** An event used in the BlockManagerUI to query information of persisted RDDs */ +private[spark] case class SparkListenerGetRDDInfo(rddInfoList: Seq[RDDInfo]) + extends SparkListenerEvent { + override def toJson = { + val rddInfoListJson = JArray(rddInfoList.map(_.toJson).toList) + super.toJson ~ + ("RDD Info List" -> rddInfoListJson) + } +} + /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent @@ -151,6 +161,7 @@ object SparkListenerEvent { val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) val loadEnvironment = Utils.getFormattedClassName(SparkListenerLoadEnvironment) val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) + val getRDDInfo = Utils.getFormattedClassName(SparkListenerGetRDDInfo) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) (json \ "Event").extract[String] match { @@ -163,32 +174,33 @@ object SparkListenerEvent { case `jobEnd` => jobEndFromJson(json) case `loadEnvironment` => loadEnvironmentFromJson(json) case `storageStatusFetch` => storageStatusFetchFromJson(json) + case `getRDDInfo` => getRDDInfoFromJson(json) case `shutdown` => SparkListenerShutdown } } - private def stageSubmittedFromJson(json: JValue) = { + private def stageSubmittedFromJson(json: JValue): SparkListenerEvent = { new SparkListenerStageSubmitted( StageInfo.fromJson(json \ "Stage Info"), Utils.propertiesFromJson(json \ "Properties")) } - private def stageCompletedFromJson(json: JValue) = { + private def stageCompletedFromJson(json: JValue): SparkListenerEvent = { new SparkListenerStageCompleted(StageInfo.fromJson(json \ "Stage Info")) } - private def taskStartFromJson(json: JValue) = { + private def taskStartFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats new SparkListenerTaskStart( (json \ "Stage ID").extract[Int], TaskInfo.fromJson(json \ "Task Info")) } - private def taskGettingResultFromJson(json: JValue) = { + private def taskGettingResultFromJson(json: JValue): SparkListenerEvent = { new SparkListenerTaskGettingResult(TaskInfo.fromJson(json \ "Task Info")) } - private def taskEndFromJson(json: JValue) = { + private def taskEndFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats new SparkListenerTaskEnd( (json \ "Stage ID").extract[Int], @@ -198,7 +210,7 @@ object SparkListenerEvent { TaskMetrics.fromJson(json \ "Task Metrics")) } - private def jobStartFromJson(json: JValue) = { + private def jobStartFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) new SparkListenerJobStart( @@ -207,14 +219,14 @@ object SparkListenerEvent { Utils.propertiesFromJson(json \ "Properties")) } - private def jobEndFromJson(json: JValue) = { + private def jobEndFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats new SparkListenerJobEnd( (json \ "Job ID").extract[Int], JobResult.fromJson(json \ "Job Result")) } - private def loadEnvironmentFromJson(json: JValue) = { + private def loadEnvironmentFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats new SparkListenerLoadEnvironment( Utils.mapFromJson(json \ "JVM Information").toSeq, @@ -223,12 +235,19 @@ object SparkListenerEvent { Utils.mapFromJson(json \ "Classpath Entries").toSeq) } - private def storageStatusFetchFromJson(json: JValue) = { + private def storageStatusFetchFromJson(json: JValue): SparkListenerEvent = { implicit val format = DefaultFormats val storageStatusList = (json \ "Storage Status List").extract[List[JValue]].map(StorageStatus.fromJson) new SparkListenerStorageStatusFetch(storageStatusList) } + + private def getRDDInfoFromJson(json: JValue): SparkListenerEvent = { + implicit val format = DefaultFormats + val rddInfoList = + (json \ "RDD Info List").extract[List[JValue]].map(RDDInfo.fromJson) + new SparkListenerGetRDDInfo(rddInfoList) + } } @@ -282,6 +301,10 @@ trait SparkListener { */ def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { } + /** + * Called when Spark queries statuses of persisted RDD's + */ + def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index fa3c3d03e2616..9f1a22b8378ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -73,6 +73,8 @@ private[spark] class SparkListenerBus extends Logging { listeners.foreach(_.onLoadEnvironment(loadEnvironment)) case storageStatusFetch: SparkListenerStorageStatusFetch => listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) + case getRDDInfo: SparkListenerGetRDDInfo => + listeners.foreach(_.onGetRDDInfo(getRDDInfo)) case SparkListenerShutdown => return true case _ => diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index af32b2eafe568..6931c13bd6976 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -76,19 +76,48 @@ case object StorageStatus { } } -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) - extends Ordered[RDDInfo] { +case class RDDInfo( + id: Int, + name: String, + storageLevel: StorageLevel, + numCachedPartitions: Int, + numPartitions: Int, + memSize: Long, + diskSize: Long) + extends JsonSerializable with Ordered[RDDInfo] { override def toString = { - import Utils.bytesToString ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, bytesToString(memSize), bytesToString(diskSize)) + numPartitions, Utils.bytesToString(memSize), Utils.bytesToString(diskSize)) } override def compare(that: RDDInfo) = { this.id - that.id } + + override def toJson = { + ("RDD ID" -> id) ~ + ("Name" -> name) ~ + ("Storage Level" -> storageLevel.toJson) ~ + ("Number of Cached Partitions" -> numCachedPartitions) ~ + ("Number of Partitions" -> numPartitions) ~ + ("Memory Size" -> memSize) ~ + ("Disk Size" -> diskSize) + } +} + +case object RDDInfo { + def fromJson(json: JValue): RDDInfo = { + implicit val format = DefaultFormats + new RDDInfo( + (json \ "RDD ID").extract[Int], + (json \ "Name").extract[String], + StorageLevel.fromJson(json \ "Storage Level"), + (json \ "Number of Cached Partitions").extract[Int], + (json \ "Number of Partitions").extract[Int], + (json \ "Memory Size").extract[Long], + (json \ "Disk Size").extract[Long]) + } } /* Helper methods for storage-related objects */ @@ -114,7 +143,7 @@ object StorageUtils { sc: SparkContext) : Array[RDDInfo] = { // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray) + val groupedRddBlocks = infos.groupBy { case (k, v) => k.rddId }.mapValues(_.values.toArray) // For each RDD, generate an RDDInfo object val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) => 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 af1fc32653617..ca5e101a1b03c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -129,7 +129,9 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends .format(dirPath)) return false } - val logFiles = logDir.listFiles.filter(_.isFile) + // Maintaining the order of log files is important because information of one job is + // dependent on that of another + val logFiles = logDir.listFiles.filter(_.isFile).sortBy(_.getName) if (logFiles.size == 0) { logWarning("No logs found in given directory %s when rendering persisted Spark Web UI!" .format(dirPath)) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 3d234a4defbdf..9bf65637bcb59 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -111,6 +111,12 @@ private[spark] class GatewayUISparkListener(live: Boolean) extends SparkListener logEvent(storageStatusFetch) logger.foreach(_.flush()) } + + override def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { + listeners.foreach(_.onGetRDDInfo(getRDDInfo)) + logEvent(getRDDInfo) + logger.foreach(_.flush()) + } } /** @@ -140,9 +146,6 @@ private[spark] class StorageStatusFetchSparkListener( } } - /** - * Update local state with fetch result, and log the appropriate event - */ override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { storageStatusList = storageStatusFetch.storageStatusList } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 547a194d58a5c..2732f18423bff 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -32,7 +32,7 @@ private[spark] object UIUtils { def prependBaseUri(resource: String = "") = uiRoot + resource /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) + def headerSparkPage(content: => Seq[Node], appName: String, title: String, page: Page.Value) : Seq[Node] = { val jobs = page match { case Stages =>
  • Stages
  • @@ -60,7 +60,7 @@ private[spark] object UIUtils { type="text/css" /> - {sc.appName} - {title} + {appName} - {title} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 924f2c03356e8..4819ce14b34ae 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -64,7 +64,7 @@ private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, sc, "Environment", Environment) + UIUtils.headerSparkPage(content, sc.appName, "Environment", Environment) } private def propertyHeader = Seq("Name", "Value") @@ -120,7 +120,6 @@ private[spark] class EnvironmentListener( } } - /** Prepare environment information for UI to render */ override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { jvmInformation = loadEnvironment.jvmInformation sparkProperties = loadEnvironment.sparkProperties diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 9dfcb50d41849..8ae10d14d64b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -72,7 +72,7 @@ private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) { ; - UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, sc.appName, "Executors (" + execInfo.size + ")", Executors) } /** Header fields for the executors table */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index abc1114e01162..7aff13f8134e7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -80,7 +80,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, sc, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, sc.appName, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index cce54fca6e168..27d688a092232 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -45,7 +45,7 @@ private[spark] class PoolPage(parent: JobProgressUI) { val content =

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, sc, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, sc.appName, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 09be0c323f696..b856ed0b12019 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -43,7 +43,8 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet - return UIUtils.headerSparkPage(content, sc, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage( + content, sc.appName, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -202,7 +203,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage(content, sc, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, sc.appName, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 2cc4d44fb78c6..ce1b7887286ce 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -22,6 +22,8 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import org.apache.spark.SparkContext +import org.apache.spark.scheduler._ +import org.apache.spark.storage.{StorageUtils, RDDInfo} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.{GatewayUISparkListener, SparkUI, StorageStatusFetchSparkListener} @@ -52,4 +54,28 @@ private[spark] class BlockManagerListener( sc: SparkContext, gateway: GatewayUISparkListener, live: Boolean) - extends StorageStatusFetchSparkListener(sc, gateway, live) + extends StorageStatusFetchSparkListener(sc, gateway, live) { + var rddInfoList: Seq[RDDInfo] = Seq() + + def getRDDInfo() { + if (live) { + val rddInfo = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + val getRDDInfo = new SparkListenerGetRDDInfo(rddInfo) + gateway.onGetRDDInfo(getRDDInfo) + } + } + + override def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { + rddInfoList = getRDDInfo.rddInfoList + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + super.onStageSubmitted(stageSubmitted) + getRDDInfo() + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { + super.onStageCompleted(stageCompleted) + getRDDInfo() + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 7ad1b3909c506..4667e4cbeb049 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{RDDInfo, StorageUtils} +import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils @@ -32,13 +32,12 @@ private[spark] class IndexPage(parent: BlockManagerUI) { private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() - val storageStatusList = listener.storageStatusList - // Calculate macro-level statistics - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + listener.fetchStorageStatus() + listener.getRDDInfo() + val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, sc, "Storage ", Storage) + UIUtils.headerSparkPage(content, sc.appName, "Storage ", Storage) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 32db30edad08a..6c2e5c239e0e8 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -27,7 +27,6 @@ import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils - /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { private val sc = parent.sc @@ -35,11 +34,12 @@ private[spark] class RDDPage(parent: BlockManagerUI) { def render(request: HttpServletRequest): Seq[Node] = { listener.fetchStorageStatus() + listener.getRDDInfo() val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList.toArray, id) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + val rddInfo = listener.rddInfoList.filter(_.id == id).head // Worker table val workers = filteredStorageStatusList.map((id, _)) @@ -96,7 +96,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage(content, sc, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, sc.appName, "RDD Storage Info for " + rddInfo.name, Storage) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 8e07a0f29addf..9451dee3c4c40 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -21,7 +21,6 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map import scala.collection.immutable -import org.apache.spark.scheduler.MapStatus import org.apache.spark.Logging /** From 6814da0cf9af2a29810b6773463acee3b259c95f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 19 Feb 2014 10:36:01 -0800 Subject: [PATCH 24/68] Explicitly register each UI listener rather than through some magic This (1) allows UISparkListener to be a simple trait and (2) is more intuitive, since it mirrors sc.addSparkListener(listener), for all other non-UI listeners. --- .../scala/org/apache/spark/ui/UISparkListener.scala | 11 +++-------- .../scala/org/apache/spark/ui/env/EnvironmentUI.scala | 6 ++++-- .../scala/org/apache/spark/ui/exec/ExecutorsUI.scala | 4 +++- .../apache/spark/ui/jobs/JobProgressListener.scala | 5 +---- .../org/apache/spark/ui/jobs/JobProgressUI.scala | 4 +++- .../org/apache/spark/ui/storage/BlockManagerUI.scala | 4 +++- 6 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 9bf65637bcb59..a48f9cba73e21 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -20,21 +20,16 @@ package org.apache.spark.ui import scala.collection.mutable.ArrayBuffer import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.SparkListenerLoadEnvironment -import org.apache.spark.scheduler.SparkListenerJobStart import org.apache.spark.SparkContext import org.apache.spark.storage.StorageStatus import org.apache.spark.util.FileLogger import net.liftweb.json.JsonAST._ -private[spark] case class UISparkListener(gateway: GatewayUISparkListener) extends SparkListener { - // Register with gateway listener - gateway.registerSparkListener(this) -} +private[spark] trait UISparkListener extends SparkListener /** - * A SparkListener that serves as a gateway for all events posted to the UI. + * A SparkListener that serves as an entry point for all events posted to the UI. * * GatewayUISparkListener achieves two functions: * @@ -130,7 +125,7 @@ private[spark] class StorageStatusFetchSparkListener( sc: SparkContext, gateway: GatewayUISparkListener, live: Boolean) - extends UISparkListener(gateway) { + extends UISparkListener { var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus /** diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 4819ce14b34ae..96207c29318f7 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -39,7 +39,9 @@ private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) { def listener = _listener.get def start() { - _listener = Some(new EnvironmentListener(sc, parent.gatewayListener, live)) + val gateway = parent.gatewayListener + _listener = Some(new EnvironmentListener(sc, gateway, live)) + gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( @@ -81,7 +83,7 @@ private[spark] class EnvironmentListener( sc: SparkContext, gateway: GatewayUISparkListener, live: Boolean) - extends UISparkListener(gateway) { + extends UISparkListener { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 8ae10d14d64b3..c7843c6a32ac7 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -39,7 +39,9 @@ private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) { def listener = _listener.get def start() { - _listener = Some(new ExecutorsListener(sc, parent.gatewayListener, live)) + val gateway = parent.gatewayListener + _listener = Some(new ExecutorsListener(sc, gateway, live)) + gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 1cfac6de8c826..051d4680242aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -31,10 +31,7 @@ import org.apache.spark.ui.{GatewayUISparkListener, UISparkListener} * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener( - sc: SparkContext, - gateway: GatewayUISparkListener) - extends UISparkListener(gateway) { +private[spark] class JobProgressListener(sc: SparkContext) extends UISparkListener { // How many stages to remember val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 6f54b099dbb23..a205ff733fc1c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -40,7 +40,9 @@ private[spark] class JobProgressUI(parent: SparkUI, live: Boolean) { def listener = _listener.get def start() { - _listener = Some(new JobProgressListener(sc, parent.gatewayListener)) + val gateway = parent.gatewayListener + _listener = Some(new JobProgressListener(sc)) + gateway.registerSparkListener(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index ce1b7887286ce..3b4169bd45a5a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -38,7 +38,9 @@ private[spark] class BlockManagerUI(parent: SparkUI, live: Boolean) { def listener = _listener.get def start() { - _listener = Some(new BlockManagerListener(sc, parent.gatewayListener, live)) + val gateway = parent.gatewayListener + _listener = Some(new BlockManagerListener(sc, gateway, live)) + gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( From d646df6786737d67d5ca1dbf593740a02a600991 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 19 Feb 2014 18:47:35 -0800 Subject: [PATCH 25/68] Completely decouple SparkUI from SparkContext This involves storing additional fields, such as the scheduling mode and the app name, into the new event, SparkListenerApplicationStart, since these attributes are no longer accessible without a SparkContext. Further, environment information is refactored to be loaded on application start (rather than on job start). Persisted Spark UI's can no longer be created from SparkContext. The new way of constructing them is through a standalone scala program. org.apache.spark.ui.UIReloader is introduced as an example of how to do this. --- .../scala/org/apache/spark/SparkConf.scala | 2 - .../scala/org/apache/spark/SparkContext.scala | 45 ++--------- .../scala/org/apache/spark/SparkEnv.scala | 52 +++++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 4 + .../org/apache/spark/scheduler/Pool.scala | 2 +- .../spark/scheduler/SparkListener.scala | 37 ++++----- .../spark/scheduler/SparkListenerBus.scala | 4 +- .../apache/spark/scheduler/TaskLocality.scala | 12 +-- .../scala/org/apache/spark/ui/SparkUI.scala | 78 ++++++++++++------- .../org/apache/spark/ui/UIReloader.scala | 46 +++++++++++ .../org/apache/spark/ui/UISparkListener.scala | 16 ++-- .../scala/org/apache/spark/ui/UIUtils.scala | 2 - .../apache/spark/ui/env/EnvironmentUI.scala | 63 +++------------ .../apache/spark/ui/exec/ExecutorsUI.scala | 9 ++- .../apache/spark/ui/jobs/ExecutorTable.scala | 4 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 60 ++++++++------ .../spark/ui/jobs/JobProgressListener.scala | 37 +++++++-- .../apache/spark/ui/jobs/JobProgressUI.scala | 8 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 9 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 6 +- .../org/apache/spark/ui/jobs/StageTable.scala | 5 +- .../spark/ui/storage/BlockManagerUI.scala | 6 +- .../apache/spark/ui/storage/IndexPage.scala | 4 +- .../org/apache/spark/ui/storage/RDDPage.scala | 4 +- 24 files changed, 297 insertions(+), 218 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/UIReloader.scala diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 45d19bcbfa6f2..b947feb891ee6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -20,8 +20,6 @@ package org.apache.spark import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} - /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f385aac3fffc0..148958542cece 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -157,9 +157,6 @@ class SparkContext( private[spark] val ui = new SparkUI(this) ui.bind() - // Keeps track of all previously persisted UI rendered by this SparkContext - private[spark] val persistedUIs = HashMap[Int, SparkUI]() - val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -211,9 +208,13 @@ class SparkContext( ui.start() + // Trigger application start + val environmentDetails = SparkEnv.environmentDetails(this) + val applicationStart = new SparkListenerApplicationStart(environmentDetails) + dagScheduler.post(applicationStart) + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { - val env = SparkEnv.get val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && @@ -579,41 +580,6 @@ class SparkContext( new CheckpointRDD[T](this, path) } - /** - * Render a previously persisted SparkUI from a set of event logs - * @param logPath Path of directory containing the event logs - */ - def renderPersistedUI(logPath: String) = { - val oldUI = new SparkUI(this, live = false) - oldUI.start() - val success = oldUI.renderFromDisk(logPath) - if (!success) { - oldUI.stop() - } else { - oldUI.bind() - persistedUIs(oldUI.boundPort.get) = oldUI - } - } - - /** - * Return a list of ports bound by persisted UI's - */ - def getPersistedUIPorts = persistedUIs.keys.toSeq - - /** - * Stop the persisted UI bound to the given port, if any - */ - def stopPersistedUI(port: Int) = { - persistedUIs.remove(port).foreach(_.stop()) - } - - /** - * Stop all persisted UI's rendered in this context - */ - def stopAllPersistedUIs() = { - persistedUIs.keys.foreach(stopPersistedUI) - } - /** Build the union of a list of RDDs. */ def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds) @@ -820,7 +786,6 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { ui.stop() - stopAllPersistedUIs() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6ae020f6a21b1..ff3f25388a631 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,10 @@ package org.apache.spark +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await +import scala.util.Properties import akka.actor._ @@ -234,4 +236,54 @@ object SparkEnv extends Logging { metricsSystem, conf) } + + /** + * Return a map representation of jvm information, Spark properties, system properties, and + * class paths. Map keys define the category, and map values represent the corresponding + * attributes as a sequence of KV pairs. + */ + private[spark] + def environmentDetails(sc: SparkContext): Map[String, Seq[(String, String)]] = { + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + + // Spark properties, including scheduling mode and app name whether or not they are configured + var additionalFields = Seq[(String, String)]() + sc.conf.getOption("spark.scheduler.mode").getOrElse { + additionalFields ++= Seq(("spark.scheduler.mode", sc.getSchedulingMode.toString)) + } + sc.conf.getOption("spark.app.name").getOrElse { + additionalFields ++= Seq(("spark.app.name", sc.appName)) + } + val sparkProperties = sc.conf.getAll.sorted ++ additionalFields + + val systemProperties = System.getProperties.iterator.toSeq + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + + // System properties that are not java classpaths + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + + // Class paths including all added jars and files + val classPathEntries = classPathProperty._2 + .split(sc.conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } + val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted + + Map[String, Seq[(String, String)]]( + "JVM Information" -> jvmInformation, + "Spark Properties" -> sparkProperties, + "System Properties" -> otherProperties, + "Classpath Entries" -> classPaths) + } } 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 24295c5a176e0..74b7e501954ba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -200,6 +200,10 @@ class DAGScheduler( listenerBus.addListener(listener) } + def post(event: SparkListenerEvent) { + listenerBus.post(event) + } + private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId] diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 4bc13c23d980b..187672c4e19e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -62,7 +62,7 @@ private[spark] class Pool( override def addSchedulable(schedulable: Schedulable) { schedulableQueue += schedulable schedulableNameToSchedulable(schedulable.name) = schedulable - schedulable.parent= this + schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { 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 0ac7462ee6aeb..09ad36b35336c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -103,18 +103,13 @@ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkLi } } -/** An event used in the EnvironmentUI */ -private[spark] case class SparkListenerLoadEnvironment( - jvmInformation: Seq[(String, String)], - sparkProperties: Seq[(String, String)], - systemProperties: Seq[(String, String)], - classpathEntries: Seq[(String, String)]) +case class SparkListenerApplicationStart(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent { override def toJson = { - val jvmInformationJson = Utils.mapToJson(jvmInformation.toMap) - val sparkPropertiesJson = Utils.mapToJson(sparkProperties.toMap) - val systemPropertiesJson = Utils.mapToJson(systemProperties.toMap) - val classpathEntriesJson = Utils.mapToJson(classpathEntries.toMap) + val jvmInformationJson = Utils.mapToJson(environmentDetails("JVM Information").toMap) + val sparkPropertiesJson = Utils.mapToJson(environmentDetails("Spark Properties").toMap) + val systemPropertiesJson = Utils.mapToJson(environmentDetails("System Properties").toMap) + val classpathEntriesJson = Utils.mapToJson(environmentDetails("Classpath Entries").toMap) super.toJson ~ ("JVM Information" -> jvmInformationJson) ~ ("Spark Properties" -> sparkPropertiesJson) ~ @@ -159,7 +154,7 @@ object SparkListenerEvent { val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) - val loadEnvironment = Utils.getFormattedClassName(SparkListenerLoadEnvironment) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) val getRDDInfo = Utils.getFormattedClassName(SparkListenerGetRDDInfo) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) @@ -172,7 +167,7 @@ object SparkListenerEvent { case `taskEnd` => taskEndFromJson(json) case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) - case `loadEnvironment` => loadEnvironmentFromJson(json) + case `applicationStart` => applicationStartFromJson(json) case `storageStatusFetch` => storageStatusFetchFromJson(json) case `getRDDInfo` => getRDDInfoFromJson(json) case `shutdown` => SparkListenerShutdown @@ -226,13 +221,13 @@ object SparkListenerEvent { JobResult.fromJson(json \ "Job Result")) } - private def loadEnvironmentFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - new SparkListenerLoadEnvironment( - Utils.mapFromJson(json \ "JVM Information").toSeq, - Utils.mapFromJson(json \ "Spark Properties").toSeq, - Utils.mapFromJson(json \ "System Properties").toSeq, - Utils.mapFromJson(json \ "Classpath Entries").toSeq) + private def applicationStartFromJson(json: JValue): SparkListenerEvent = { + val environmentDetails = Map[String, Seq[(String, String)]]( + "JVM Information" -> Utils.mapFromJson(json \ "JVM Information").toSeq, + "Spark Properties" -> Utils.mapFromJson(json \ "Spark Properties").toSeq, + "System Properties" -> Utils.mapFromJson(json \ "System Properties").toSeq, + "Classpath Entries" -> Utils.mapFromJson(json \ "Classpath Entries").toSeq) + new SparkListenerApplicationStart(environmentDetails) } private def storageStatusFetchFromJson(json: JValue): SparkListenerEvent = { @@ -292,9 +287,9 @@ trait SparkListener { def onJobEnd(jobEnd: SparkListenerJobEnd) { } /** - * Called when the Spark environment is loaded + * Called when the application starts */ - def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { } + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } /** * Called when Spark fetches storage statuses from the driver diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 9f1a22b8378ee..2eb57996b4feb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -69,8 +69,8 @@ private[spark] class SparkListenerBus extends Logging { listeners.foreach(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => listeners.foreach(_.onTaskEnd(taskEnd)) - case loadEnvironment: SparkListenerLoadEnvironment => - listeners.foreach(_.onLoadEnvironment(loadEnvironment)) + case applicationStart: SparkListenerApplicationStart => + listeners.foreach(_.onApplicationStart(applicationStart)) case storageStatusFetch: SparkListenerStorageStatusFetch => listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) case getRDDInfo: SparkListenerGetRDDInfo => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 7cc39b7a65b8d..766ce0cfd1e1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -32,16 +32,6 @@ private[spark] object TaskLocality extends Enumeration { def fromJson(json: JValue): TaskLocality = { implicit val format = DefaultFormats - val processLocal = PROCESS_LOCAL.toString - val nodeLocal = NODE_LOCAL.toString - val rackLocal = RACK_LOCAL.toString - val any = ANY.toString - - json.extract[String] match { - case `processLocal` => PROCESS_LOCAL - case `nodeLocal` => NODE_LOCAL - case `rackLocal` => RACK_LOCAL - case `any` => ANY - } + TaskLocality.withName(json.extract[String]) } } 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 ca5e101a1b03c..a67f962de6a60 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -36,35 +36,37 @@ import net.liftweb.json._ import it.unimi.dsi.fastutil.io.FastBufferedInputStream -/** Top level user interface for Spark */ -private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends Logging { +/** Top level user interface for Spark. */ +private[spark] class SparkUI(val sc: SparkContext) extends Logging { + + // If SparkContext is not provided, assume this UI is rendered from persisted storage + val live = sc != null val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = if (live) { + var port = if (live) { sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt } else { - // While each context has only one live SparkUI, it can have many persisted ones - // For persisted UI's, climb upwards from the configured / default port - val nextPort = SparkUI.lastPersistedUIPort match { - case Some(p) => p + 1 - case None => sc.conf.get("spark.ui.persisted.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt - } - SparkUI.lastPersistedUIPort = Some(nextPort) - nextPort + SparkUI.DEFAULT_PERSISTED_PORT.toInt } var boundPort: Option[Int] = None var server: Option[Server] = None + var started = false + var appName = "" private val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("/", createRedirectHandler("/stages")) ) - private val storage = new BlockManagerUI(this, live) - private val jobs = new JobProgressUI(this, live) - private val env = new EnvironmentUI(this, live) - private val exec = new ExecutorsUI(this, live) + private val storage = new BlockManagerUI(this) + private val jobs = new JobProgressUI(this) + private val env = new EnvironmentUI(this) + private val exec = new ExecutorsUI(this) // Add MetricsServlet handlers by default - private val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers + private val metricsServletHandlers = if (live) { + SparkEnv.get.metricsSystem.getServletHandlers + } else { + Array[(String, Handler)]() + } private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers @@ -73,11 +75,23 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends private var _gatewayListener: Option[GatewayUISparkListener] = None def gatewayListener = _gatewayListener.getOrElse { - val gateway = new GatewayUISparkListener(live) + val gateway = new GatewayUISparkListener(this, live) _gatewayListener = Some(gateway) gateway } + // Only meaningful if port is set before binding + def setPort(p: Int) = { + if (boundPort.isDefined) { + logWarning("Attempted to set Spark Web UI port after it is already bound to %s." + .format(appUIAddress)) + } else { + port = p + } + } + + def setAppName(name: String) = appName = name + /** Bind the HTTP server which backs this web interface */ def bind() { try { @@ -107,6 +121,7 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends jobs.start() env.start() exec.start() + started = true } def stop() { @@ -116,11 +131,13 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends /** * Reconstruct a previously persisted SparkUI from logs residing in the given directory. - * Return true if log files are found and processed. + * + * This method must be invoked after the SparkUI has started. Return true if log files + * are found and processed. */ - def renderFromDisk(dirPath: String): Boolean = { - // Live UI's should never invoke this - assert(!live) + def renderFromPersistedStorage(dirPath: String): Boolean = { + assert(!live, "Live Spark Web UI attempted to render from persisted storage!") + assert(started, "Spark Web UI attempted to render from persisted storage before starting!") // Check validity of the given path val logDir = new File(dirPath) @@ -129,8 +146,7 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends .format(dirPath)) return false } - // Maintaining the order of log files is important because information of one job is - // dependent on that of another + // Assume events are ordered not only within each log file, but also across files by file name val logFiles = logDir.listFiles.filter(_.isFile).sortBy(_.getName) if (logFiles.size == 0) { logWarning("No logs found in given directory %s when rendering persisted Spark Web UI!" @@ -139,14 +155,19 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends } // Replay events in each event log - logFiles.foreach(processEventLog) + // Use a new SparkListenerBus to avoid depending on SparkContext + val bus = new SparkListenerBus + logFiles.foreach { file => processEventLog(file, bus) } true } /** - * Replay each event in the order maintained in the given log to the gateway listener + * Replay each event in the order maintained in the given log to the gateway listener. + * + * A custom SparkListenerBus, rather than the DAG scheduler's, is used to decouple the + * replaying of logged events from the creation of a SparkContext. */ - private def processEventLog(file: File) = { + private def processEventLog(file: File, listenerBus: SparkListenerBus) = { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) var currentLine = "" @@ -155,7 +176,7 @@ private[spark] class SparkUI(val sc: SparkContext, live: Boolean = true) extends lines.foreach { line => currentLine = line val event = SparkListenerEvent.fromJson(parse(line)) - sc.dagScheduler.listenerBus.postToListeners(event, Seq(gatewayListener)) + listenerBus.postToListeners(event, Seq(gatewayListener)) } } catch { case e: Exception => @@ -174,7 +195,4 @@ private[spark] object SparkUI { val DEFAULT_PORT = "4040" val DEFAULT_PERSISTED_PORT = "14040" val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" - - // Keep track of the port of the last persisted UI - var lastPersistedUIPort: Option[Int] = None } diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala new file mode 100644 index 0000000000000..c6199967e6ffc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -0,0 +1,46 @@ +/* + * 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 + +/** + * Reload a persisted UI independently from a SparkContext + */ +object UIReloader { + def main(args: Array[String]) { + if (args.length < 1) { + println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path]") + System.exit(1) + } + + val ui = new SparkUI(null) + ui.bind() + ui.start() + val success = ui.renderFromPersistedStorage(args(0)) + if (!success) { + ui.stop() + } + + println("\nTo exit, type exit or quit.") + var line = "" + while (line != "exit" && line != "quit") { + print("> ") + line = readLine() + } + println("\nReceived signal to exit.") + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index a48f9cba73e21..0cf0862ee325e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -41,7 +41,7 @@ private[spark] trait UISparkListener extends SparkListener * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized * from the event logs to all attached listeners. */ -private[spark] class GatewayUISparkListener(live: Boolean) extends SparkListener { +private[spark] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends SparkListener { // Log events only if the UI is live private val logger: Option[FileLogger] = if (live) Some(new FileLogger()) else None @@ -95,9 +95,15 @@ private[spark] class GatewayUISparkListener(live: Boolean) extends SparkListener logger.foreach(_.close()) } - override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { - listeners.foreach(_.onLoadEnvironment(loadEnvironment)) - logEvent(loadEnvironment) + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + // Retrieve app name from the application start event + // For live UI's, this should be equivalent to sc.appName + val sparkProperties = applicationStart.environmentDetails("Spark Properties").toMap + val appName = sparkProperties.get("spark.app.name") + appName.foreach(parent.setAppName) + + listeners.foreach(_.onApplicationStart(applicationStart)) + logEvent(applicationStart) logger.foreach(_.flush()) } @@ -126,7 +132,7 @@ private[spark] class StorageStatusFetchSparkListener( gateway: GatewayUISparkListener, live: Boolean) extends UISparkListener { - var storageStatusList: Seq[StorageStatus] = sc.getExecutorStorageStatus + var storageStatusList: Seq[StorageStatus] = Seq() /** * Fetch storage information from SparkEnv, which involves a query to the driver. This is diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 2732f18423bff..c1f31cedd8bf0 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -19,8 +19,6 @@ package org.apache.spark.ui import scala.xml.Node -import org.apache.spark.SparkContext - /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { import Page._ diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 96207c29318f7..3cc85f6629b76 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -19,8 +19,6 @@ package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest -import scala.collection.JavaConversions._ -import scala.util.Properties import scala.xml.Node import org.eclipse.jetty.server.Handler @@ -31,16 +29,18 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment import org.apache.spark.ui._ -private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) { +private[spark] class EnvironmentUI(parent: SparkUI) { + val live = parent.live val sc = parent.sc private var _listener: Option[EnvironmentListener] = None + def appName = parent.appName def listener = _listener.get def start() { val gateway = parent.gatewayListener - _listener = Some(new EnvironmentListener(sc, gateway, live)) + _listener = Some(new EnvironmentListener(sc)) gateway.registerSparkListener(listener) } @@ -49,7 +49,6 @@ private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) { ) def render(request: HttpServletRequest): Seq[Node] = { - listener.loadEnvironment() val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( @@ -66,7 +65,7 @@ private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, sc.appName, "Environment", Environment) + UIUtils.headerSparkPage(content, appName, "Environment", Environment) } private def propertyHeader = Seq("Name", "Value") @@ -79,55 +78,17 @@ private[spark] class EnvironmentUI(parent: SparkUI, live: Boolean) { /** * A SparkListener that prepares information to be displayed on the EnvironmentUI */ -private[spark] class EnvironmentListener( - sc: SparkContext, - gateway: GatewayUISparkListener, - live: Boolean) - extends UISparkListener { +private[spark] class EnvironmentListener(sc: SparkContext) extends UISparkListener { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() - /** Gather JVM, spark, system and classpath properties */ - def loadEnvironment() { - if (live) { - // Load environment only this is a live UI - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - val sparkProperties = sc.conf.getAll.sorted - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted - - // Trigger SparkListenerLoadEnvironment - val loadEnvironment = new SparkListenerLoadEnvironment( - jvmInformation, sparkProperties, otherProperties, classPaths) - gateway.onLoadEnvironment(loadEnvironment) - } - } - - override def onLoadEnvironment(loadEnvironment: SparkListenerLoadEnvironment) { - jvmInformation = loadEnvironment.jvmInformation - sparkProperties = loadEnvironment.sparkProperties - systemProperties = loadEnvironment.systemProperties - classpathEntries = loadEnvironment.classpathEntries + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + val environmentDetails = applicationStart.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") } - - override def onJobStart(jobStart: SparkListenerJobStart) = loadEnvironment() } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index c7843c6a32ac7..13e332e1e20a0 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -31,11 +31,13 @@ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui._ import org.apache.spark.util.Utils -private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) { +private[spark] class ExecutorsUI(parent: SparkUI) { + val live = parent.live val sc = parent.sc private var _listener: Option[ExecutorsListener] = None + def appName = parent.appName def listener = _listener.get def start() { @@ -55,7 +57,8 @@ private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) { val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) - val execTable = UIUtils.listingTable(execHeader, execRow, execInfo) + val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) + val execTable = UIUtils.listingTable(execHeader, execRow, execInfoSorted) val content =
    @@ -74,7 +77,7 @@ private[spark] class ExecutorsUI(parent: SparkUI, live: Boolean) {
    ; - UIUtils.headerSparkPage(content, sc.appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, appName, "Executors (" + execInfo.size + ")", Executors) } /** Header fields for the executors table */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index deaeb8a5e0dae..a37bc7f9aeb9d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -56,9 +56,9 @@ private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { private def createExecutorTable() : Seq[Node] = { // Make an executor-id -> address map val executorIdToAddress = mutable.HashMap[String, String]() - val storageStatusList = parent.sc.getExecutorStorageStatus + val storageStatusList = listener.storageStatusList for (statusId <- 0 until storageStatusList.size) { - val blockManagerId = parent.sc.getExecutorStorageStatus(statusId).blockManagerId + val blockManagerId = storageStatusList(statusId).blockManagerId val address = blockManagerId.hostPort val executorId = blockManagerId.executorId executorIdToAddress.put(executorId, address) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 7aff13f8134e7..f753741cb26ac 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -21,13 +21,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{NodeSeq, Node} -import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { + private val live = parent.live private val sc = parent.sc + private def appName = parent.appName + private def isFairScheduler = parent.isFairScheduler private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -42,34 +44,42 @@ private[spark] class IndexPage(parent: JobProgressUI) { new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val pools = sc.getAllPools + // For now, pool information is only accessible in live UI's + val pools = if (live) sc.getAllPools else Seq() val poolTable = new PoolTable(pools, parent) + val summary: NodeSeq = -
    -
      -
    • - Total Duration: - {parent.formatDuration(now - sc.startTime)} -
    • -
    • Scheduling Mode: {sc.getSchedulingMode}
    • -
    • - Active Stages: - {activeStages.size} -
    • -
    • - Completed Stages: - {completedStages.size} -
    • -
    • +
      +
        + {if (live) { + // Total duration is not meaningful unless the UI is live +
      • + Total Duration: + {parent.formatDuration(now - sc.startTime)} +
      • + }} +
      • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
      • +
      • + Active Stages: + {activeStages.size} +
      • +
      • + Completed Stages: + {completedStages.size} +
      • +
      • Failed Stages: - {failedStages.size} -
      • -
      -
      + {failedStages.size} +
    • +
    +
    val content = summary ++ - {if (sc.getSchedulingMode == SchedulingMode.FAIR) { -

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq + {if (live && isFairScheduler) { +

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq } else { Seq() }} ++ @@ -80,7 +90,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, sc.appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, appName, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 051d4680242aa..e38bbc091d128 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -22,7 +22,8 @@ import scala.collection.mutable.{ListBuffer, HashMap} import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.ui.{GatewayUISparkListener, UISparkListener} +import org.apache.spark.ui.{GatewayUISparkListener, StorageStatusFetchSparkListener} +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** * Tracks task-level information to be displayed in the UI. @@ -31,11 +32,19 @@ import org.apache.spark.ui.{GatewayUISparkListener, UISparkListener} * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(sc: SparkContext) extends UISparkListener { +private[spark] class JobProgressListener( + sc: SparkContext, + gateway: GatewayUISparkListener, + live: Boolean) + extends StorageStatusFetchSparkListener(sc, gateway, live) { + import JobProgressListener._ // How many stages to remember - val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) - val DEFAULT_POOL_NAME = "default" + val retainedStages = if (live) { + sc.conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) + } else { + DEFAULT_RETAINED_STAGES + } val stageIdToPool = new HashMap[Int, String]() val stageIdToDescription = new HashMap[Int, String]() @@ -61,6 +70,8 @@ private[spark] class JobProgressListener(sc: SparkContext) extends UISparkListen val stageIdToTaskInfos = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() + var schedulingMode: Option[SchedulingMode] = None + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo val stageId = stage.stageId @@ -73,8 +84,8 @@ private[spark] class JobProgressListener(sc: SparkContext) extends UISparkListen /** If stages is too large, remove and garbage collect old stages */ private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { - if (stages.size > RETAINED_STAGES) { - val toRemove = RETAINED_STAGES / 10 + if (stages.size > retainedStages) { + val toRemove = retainedStages / 10 stages.takeRight(toRemove).foreach( s => { stageIdToTaskInfos.remove(s.stageId) stageIdToTime.remove(s.stageId) @@ -218,9 +229,23 @@ private[spark] class JobProgressListener(sc: SparkContext) extends UISparkListen case _ => } } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) = synchronized { + val schedulingModeName = + applicationStart.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") + schedulingMode = schedulingModeName match { + case Some(name) => Some(SchedulingMode.withName(name)) + case None => None + } + } } private[spark] case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, exception: Option[ExceptionFailure] = None) + +private[spark] object JobProgressListener { + val DEFAULT_RETAINED_STAGES = 1000 + val DEFAULT_POOL_NAME = "default" +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index a205ff733fc1c..a0e681c777198 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -23,13 +23,15 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(parent: SparkUI, live: Boolean) { +private[spark] class JobProgressUI(parent: SparkUI) { val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val live = parent.live val sc = parent.sc private val indexPage = new IndexPage(this) @@ -37,11 +39,13 @@ private[spark] class JobProgressUI(parent: SparkUI, live: Boolean) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None + def appName = parent.appName + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) def listener = _listener.get def start() { val gateway = parent.gatewayListener - _listener = Some(new JobProgressListener(sc)) + _listener = Some(new JobProgressListener(sc, gateway, live)) gateway.registerSparkListener(listener) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 27d688a092232..29bef73ed9b50 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,7 +26,9 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { + private val live = parent.live private val sc = parent.sc + private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -39,13 +41,14 @@ private[spark] class PoolPage(parent: JobProgressUI) { } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val pool = sc.getPoolForName(poolName).get - val poolTable = new PoolTable(Seq(pool), parent) + // For now, pool information is only accessible in live UI's + val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq() + val poolTable = new PoolTable(pools, parent) val content =

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, sc.appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, appName, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b856ed0b12019..37e7e6302599f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -29,8 +29,8 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI) { - private val sc = parent.sc private val dateFmt = parent.dateFmt + private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -44,7 +44,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Tasks

    No tasks have started yet return UIUtils.headerSparkPage( - content, sc.appName, "Details for Stage %s".format(stageId), Stages) + content, appName, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -203,7 +203,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage(content, sc.appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, appName, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5b4027f90172a..3983bb00184d5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -22,15 +22,14 @@ import java.util.Date import scala.xml.Node import scala.collection.mutable.HashMap -import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} +import org.apache.spark.scheduler.{StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { - private val sc = parent.sc private val dateFmt = parent.dateFmt - private val isFairScheduler = sc.getSchedulingMode == SchedulingMode.FAIR + private def isFairScheduler = parent.isFairScheduler private def listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 3b4169bd45a5a..a9c2d956f0b6a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -25,16 +25,18 @@ import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageUtils, RDDInfo} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.{GatewayUISparkListener, SparkUI, StorageStatusFetchSparkListener} +import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(parent: SparkUI, live: Boolean) { +private[spark] class BlockManagerUI(parent: SparkUI) { + val live = parent.live val sc = parent.sc private val indexPage = new IndexPage(this) private val rddPage = new RDDPage(this) private var _listener: Option[BlockManagerListener] = None + def appName = parent.appName def listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 4667e4cbeb049..3c4cb119fc05b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { - private val sc = parent.sc + private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -37,7 +37,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { listener.getRDDInfo() val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, sc.appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, appName, "Storage ", Storage) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 6c2e5c239e0e8..0f8ac552ab052 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { - private val sc = parent.sc + private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -96,7 +96,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage(content, sc.appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, appName, "RDD Storage Info for " + rddInfo.name, Storage) } /** Header fields for the worker table */ From e9e1c6dede36788d3cefe3c65366f5a79be97a1d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 20 Feb 2014 23:51:08 -0800 Subject: [PATCH 26/68] Move all JSON de/serialization logic to JsonProtocol This makes all classes involved appear less cluttered. --- .../org/apache/spark/TaskEndReason.scala | 84 +- .../deploy/master/ui/ApplicationPage.scala | 4 +- .../spark/deploy/master/ui/IndexPage.scala | 4 +- .../spark/deploy/worker/ui/IndexPage.scala | 4 +- .../apache/spark/executor/TaskMetrics.scala | 94 +-- .../apache/spark/scheduler/JobResult.scala | 39 +- .../spark/scheduler/SparkListener.scala | 214 +----- .../apache/spark/scheduler/StageInfo.scala | 68 +- .../org/apache/spark/scheduler/TaskInfo.scala | 40 +- .../apache/spark/scheduler/TaskLocality.scala | 8 - .../org/apache/spark/storage/BlockId.scala | 128 +--- .../apache/spark/storage/BlockManagerId.scala | 23 +- .../storage/BlockManagerMasterActor.scala | 22 - .../apache/spark/storage/StorageLevel.scala | 24 +- .../apache/spark/storage/StorageUtils.scala | 60 +- .../scala/org/apache/spark/ui/SparkUI.scala | 3 +- .../org/apache/spark/ui/UISparkListener.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 725 ++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 85 +- .../spark/deploy/JsonProtocolSuite.scala | 1 + 20 files changed, 782 insertions(+), 852 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/JsonProtocol.scala diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 87733d66e51bf..f1a753b6ab8a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -19,106 +19,32 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId -import org.apache.spark.scheduler.JsonSerializable -import org.apache.spark.util.Utils - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats /** * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason extends JsonSerializable { - override def toJson = "Reason" -> Utils.getFormattedClassName(this) -} - -private[spark] case object TaskEndReason { - def fromJson(json: JValue): TaskEndReason = { - implicit val format = DefaultFormats - val success = Utils.getFormattedClassName(Success) - val resubmitted = Utils.getFormattedClassName(Resubmitted) - val fetchFailed = Utils.getFormattedClassName(FetchFailed) - val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) - val taskResultLost = Utils.getFormattedClassName(TaskResultLost) - val taskKilled = Utils.getFormattedClassName(TaskKilled) - val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) - val unknownReason = Utils.getFormattedClassName(UnknownReason) - - (json \ "Reason").extract[String] match { - case `success` => Success - case `resubmitted` => Resubmitted - case `fetchFailed` => fetchFailedFromJson(json) - case `exceptionFailure` => exceptionFailureFromJson(json) - case `taskResultLost` => TaskResultLost - case `taskKilled` => TaskKilled - case `executorLostFailure` => ExecutorLostFailure - case `unknownReason` => UnknownReason - } - } - - private def fetchFailedFromJson(json: JValue): TaskEndReason = { - implicit val format = DefaultFormats - new FetchFailed( - BlockManagerId.fromJson(json \ "Block Manager Address"), - (json \ "Shuffle ID").extract[Int], - (json \ "Map ID").extract[Int], - (json \ "Reduce ID").extract[Int]) - } - - private def exceptionFailureFromJson(json: JValue): TaskEndReason = { - implicit val format = DefaultFormats - val metrics = (json \ "Metrics") match { - case JNothing => None - case value: JValue => Some(TaskMetrics.fromJson(value)) - } - val stackTrace = Utils.stackTraceFromJson(json \ "Stack Trace") - new ExceptionFailure( - (json \ "Class Name").extract[String], - (json \ "Description").extract[String], - stackTrace, - metrics) - } -} +private[spark] sealed trait TaskEndReason private[spark] case object Success extends TaskEndReason -// Task was finished earlier but we've now lost it -private[spark] case object Resubmitted extends TaskEndReason +private[spark] +case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it private[spark] case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) - extends TaskEndReason { - override def toJson = { - super.toJson ~ - ("Block Manager Address" -> bmAddress.toJson) ~ - ("Shuffle ID" -> shuffleId) ~ - ("Map ID" -> mapId) ~ - ("Reduce ID" -> reduceId) - } -} + extends TaskEndReason private[spark] case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], metrics: Option[TaskMetrics]) - extends TaskEndReason { - override def toJson = { - val stackTraceJson = Utils.stackTraceToJson(stackTrace) - val metricsJson = metrics.map(_.toJson).getOrElse(JNothing) - super.toJson ~ - ("Class Name" -> className) ~ - ("Description" -> description) ~ - ("Stack Trace" -> stackTraceJson) ~ - ("Metrics" -> metricsJson) - } -} + extends TaskEndReason /** * The task finished successfully, but the result was lost from the executor's block manager before diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index f29a6ad2e7b92..690fc4654a054 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -24,11 +24,11 @@ import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils} +import org.apache.spark.deploy.JsonProtocol private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 04f9a22a25a94..6b1e1a77af290 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -25,11 +25,11 @@ import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} +import org.apache.spark.deploy.{JsonProtocol, DeployWebUI} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.ui.UIUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils} private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 3089acffb8d98..602e38e8733f2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -24,12 +24,12 @@ import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.UIUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils} +import org.apache.spark.deploy.JsonProtocol private[spark] class IndexPage(parent: WorkerWebUI) { val workerActor = parent.worker.self diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e6196ad554532..4aa51d0426a99 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,13 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.scheduler.JsonSerializable - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats - -class TaskMetrics extends Serializable with JsonSerializable { +class TaskMetrics extends Serializable { /** * Host's name the task runs on */ @@ -74,22 +68,13 @@ class TaskMetrics extends Serializable with JsonSerializable { * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None +} - override def toJson = { - ("Host Name" -> hostname) ~ - ("Executor Deserialize Time" -> executorDeserializeTime) ~ - ("Executor Run Time" -> executorRunTime) ~ - ("Result Size" -> resultSize) ~ - ("JVM GC Time" -> jvmGCTime) ~ - ("Result Serialization Time" -> resultSerializationTime) ~ - ("Memory Bytes Spilled" -> memoryBytesSpilled) ~ - ("Disk Bytes Spilled" -> diskBytesSpilled) ~ - ("Shuffle Read Metrics" -> shuffleReadMetrics.map(_.toJson).getOrElse(JNothing)) ~ - ("Shuffle Write Metrics" -> shuffleWriteMetrics.map(_.toJson).getOrElse(JNothing)) - } +object TaskMetrics { + private[spark] def empty(): TaskMetrics = new TaskMetrics } -class ShuffleReadMetrics extends Serializable with JsonSerializable { +class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data */ @@ -128,19 +113,9 @@ class ShuffleReadMetrics extends Serializable with JsonSerializable { * Total number of remote bytes read from the shuffle by this task */ var remoteBytesRead: Long = _ - - override def toJson = { - ("Shuffle Finish Time" -> shuffleFinishTime) ~ - ("Total Blocks Fetched" -> totalBlocksFetched) ~ - ("Remote Blocks Fetched" -> remoteBlocksFetched) ~ - ("Local Blocks Fetched" -> localBlocksFetched) ~ - ("Fetch Wait Time" -> fetchWaitTime) ~ - ("Remote Fetch Time" -> remoteFetchTime) ~ - ("Remote Bytes Read" -> remoteBytesRead) - } } -class ShuffleWriteMetrics extends Serializable with JsonSerializable { +class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task */ @@ -150,61 +125,4 @@ class ShuffleWriteMetrics extends Serializable with JsonSerializable { * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ var shuffleWriteTime: Long = _ - - override def toJson = { - ("Shuffle Bytes Written" -> shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteTime) - } -} - -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics - - def fromJson(json: JValue): TaskMetrics = { - implicit val format = DefaultFormats - val metrics = new TaskMetrics - metrics.hostname = (json \ "Host Name").extract[String] - metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] - metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] - metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] - metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] - metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] - metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] - metrics.shuffleReadMetrics = - json \ "Shuffle Read Metrics" match { - case JNothing => None - case value: JValue => Some(ShuffleReadMetrics.fromJson(value)) - } - metrics.shuffleWriteMetrics = - json \ "Shuffle Write Metrics" match { - case JNothing => None - case value: JValue => Some(ShuffleWriteMetrics.fromJson(value)) - } - metrics - } -} - -object ShuffleReadMetrics { - def fromJson(json: JValue): ShuffleReadMetrics = { - implicit val format = DefaultFormats - val metrics = new ShuffleReadMetrics - metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] - metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] - metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] - metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] - metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] - metrics.remoteFetchTime = (json \ "Remote Fetch Time").extract[Long] - metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] - metrics - } -} - -object ShuffleWriteMetrics { - def fromJson(json: JValue): ShuffleWriteMetrics = { - implicit val format = DefaultFormats - val metrics = new ShuffleWriteMetrics - metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] - metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] - metrics - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 71023a7e194de..3cf4e3077e4a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,47 +17,12 @@ package org.apache.spark.scheduler -import org.apache.spark.util.Utils - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats - /** * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult extends JsonSerializable { - override def toJson = "Result" -> Utils.getFormattedClassName(this) -} - -private[spark] object JobResult { - def fromJson(json: JValue): JobResult = { - implicit val format = DefaultFormats - val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) - val jobFailed = Utils.getFormattedClassName(JobFailed) - - (json \ "Result").extract[String] match { - case `jobSucceeded` => JobSucceeded - case `jobFailed` => jobFailedFromJson(json) - } - } - - private def jobFailedFromJson(json: JValue): JobResult = { - implicit val format = DefaultFormats - new JobFailed( - Utils.exceptionFromJson(json \ "Exception"), - (json \ "Failed Stage ID").extract[Int]) - } -} +private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult // A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult { - override def toJson = { - val exceptionJson = Utils.exceptionToJson(exception) - super.toJson ~ - ("Exception" -> exceptionJson) ~ - ("Failed Stage ID" -> failedStageId) - } -} +private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult 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 09ad36b35336c..a492cfc07cd78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,232 +19,46 @@ package org.apache.spark.scheduler import java.util.Properties +import scala.collection.Map + import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.{RDDInfo, StorageStatus} -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats - -trait JsonSerializable { def toJson: JValue } - -sealed trait SparkListenerEvent extends JsonSerializable { - override def toJson = "Event" -> Utils.getFormattedClassName(this) -} +sealed trait SparkListenerEvent case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties) - extends SparkListenerEvent { - override def toJson = { - val propertiesJson = Utils.propertiesToJson(properties) - super.toJson ~ - ("Stage Info" -> stageInfo.toJson) ~ - ("Properties" -> propertiesJson) - } -} + extends SparkListenerEvent -case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent { - override def toJson = { - super.toJson ~ - ("Stage Info" -> stageInfo.toJson) - } -} +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent -case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent { - override def toJson = { - super.toJson ~ - ("Stage ID" -> stageId) ~ - ("Task Info" -> taskInfo.toJson) - } -} +case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent { - override def toJson = { - super.toJson ~ - ("Task Info" -> taskInfo.toJson) - } -} +case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerTaskEnd( - stageId: Int, - taskType: String, - reason: TaskEndReason, - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) - extends SparkListenerEvent { - override def toJson = { - super.toJson ~ - ("Stage ID" -> stageId) ~ - ("Task Type" -> taskType) ~ - ("Task End Reason" -> reason.toJson) ~ - ("Task Info" -> taskInfo.toJson) ~ - ("Task Metrics" -> taskMetrics.toJson) - } -} +case class SparkListenerTaskEnd(stageId: Int, taskType: String, reason: TaskEndReason, + taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvent case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) - extends SparkListenerEvent { - override def toJson = { - val stageIdsJson = JArray(stageIds.map(JInt(_)).toList) - val propertiesJson = Utils.propertiesToJson(properties) - super.toJson ~ - ("Job ID" -> jobId) ~ - ("Stage IDs" -> stageIdsJson) ~ - ("Properties" -> propertiesJson) - } -} + extends SparkListenerEvent -case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent { - override def toJson = { - super.toJson ~ - ("Job ID" -> jobId) ~ - ("Job Result" -> jobResult.toJson) - } -} +case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent case class SparkListenerApplicationStart(environmentDetails: Map[String, Seq[(String, String)]]) - extends SparkListenerEvent { - override def toJson = { - val jvmInformationJson = Utils.mapToJson(environmentDetails("JVM Information").toMap) - val sparkPropertiesJson = Utils.mapToJson(environmentDetails("Spark Properties").toMap) - val systemPropertiesJson = Utils.mapToJson(environmentDetails("System Properties").toMap) - val classpathEntriesJson = Utils.mapToJson(environmentDetails("Classpath Entries").toMap) - super.toJson ~ - ("JVM Information" -> jvmInformationJson) ~ - ("Spark Properties" -> sparkPropertiesJson) ~ - ("System Properties" -> systemPropertiesJson) ~ - ("Classpath Entries" -> classpathEntriesJson) - } -} + extends SparkListenerEvent /** An event used in the ExecutorsUI and BlockManagerUI to fetch storage status from SparkEnv */ private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq[StorageStatus]) - extends SparkListenerEvent { - override def toJson = { - val storageStatusListJson = JArray(storageStatusList.map(_.toJson).toList) - super.toJson ~ - ("Storage Status List" -> storageStatusListJson) - } -} + extends SparkListenerEvent /** An event used in the BlockManagerUI to query information of persisted RDDs */ private[spark] case class SparkListenerGetRDDInfo(rddInfoList: Seq[RDDInfo]) - extends SparkListenerEvent { - override def toJson = { - val rddInfoListJson = JArray(rddInfoList.map(_.toJson).toList) - super.toJson ~ - ("RDD Info List" -> rddInfoListJson) - } -} + extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent -object SparkListenerEvent { - /** - * Deserialize a SparkListenerEvent from JSON - */ - def fromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) - val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) - val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) - val taskGettingResult = Utils.getFormattedClassName(SparkListenerTaskGettingResult) - val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) - val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) - val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) - val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) - val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) - val getRDDInfo = Utils.getFormattedClassName(SparkListenerGetRDDInfo) - val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) - - (json \ "Event").extract[String] match { - case `stageSubmitted` => stageSubmittedFromJson(json) - case `stageCompleted` => stageCompletedFromJson(json) - case `taskStart` => taskStartFromJson(json) - case `taskGettingResult` => taskGettingResultFromJson(json) - case `taskEnd` => taskEndFromJson(json) - case `jobStart` => jobStartFromJson(json) - case `jobEnd` => jobEndFromJson(json) - case `applicationStart` => applicationStartFromJson(json) - case `storageStatusFetch` => storageStatusFetchFromJson(json) - case `getRDDInfo` => getRDDInfoFromJson(json) - case `shutdown` => SparkListenerShutdown - } - } - - private def stageSubmittedFromJson(json: JValue): SparkListenerEvent = { - new SparkListenerStageSubmitted( - StageInfo.fromJson(json \ "Stage Info"), - Utils.propertiesFromJson(json \ "Properties")) - } - - private def stageCompletedFromJson(json: JValue): SparkListenerEvent = { - new SparkListenerStageCompleted(StageInfo.fromJson(json \ "Stage Info")) - } - - private def taskStartFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - new SparkListenerTaskStart( - (json \ "Stage ID").extract[Int], - TaskInfo.fromJson(json \ "Task Info")) - } - - private def taskGettingResultFromJson(json: JValue): SparkListenerEvent = { - new SparkListenerTaskGettingResult(TaskInfo.fromJson(json \ "Task Info")) - } - - private def taskEndFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - new SparkListenerTaskEnd( - (json \ "Stage ID").extract[Int], - (json \ "Task Type").extract[String], - TaskEndReason.fromJson(json \ "Task End Reason"), - TaskInfo.fromJson(json \ "Task Info"), - TaskMetrics.fromJson(json \ "Task Metrics")) - } - - private def jobStartFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) - new SparkListenerJobStart( - (json \ "Job ID").extract[Int], - stageIds, - Utils.propertiesFromJson(json \ "Properties")) - } - - private def jobEndFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - new SparkListenerJobEnd( - (json \ "Job ID").extract[Int], - JobResult.fromJson(json \ "Job Result")) - } - - private def applicationStartFromJson(json: JValue): SparkListenerEvent = { - val environmentDetails = Map[String, Seq[(String, String)]]( - "JVM Information" -> Utils.mapFromJson(json \ "JVM Information").toSeq, - "Spark Properties" -> Utils.mapFromJson(json \ "Spark Properties").toSeq, - "System Properties" -> Utils.mapFromJson(json \ "System Properties").toSeq, - "Classpath Entries" -> Utils.mapFromJson(json \ "Classpath Entries").toSeq) - new SparkListenerApplicationStart(environmentDetails) - } - - private def storageStatusFetchFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - val storageStatusList = - (json \ "Storage Status List").extract[List[JValue]].map(StorageStatus.fromJson) - new SparkListenerStorageStatusFetch(storageStatusList) - } - - private def getRDDInfoFromJson(json: JValue): SparkListenerEvent = { - implicit val format = DefaultFormats - val rddInfoList = - (json \ "RDD Info List").extract[List[JValue]].map(RDDInfo.fromJson) - new SparkListenerGetRDDInfo(rddInfoList) - } -} - /** * Interface for listening to events from the Spark scheduler. diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 228ab6162efee..6cfe34e836f74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -21,14 +21,11 @@ import scala.collection.mutable import org.apache.spark.executor.TaskMetrics -import net.liftweb.json.JsonAST._ -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.DefaultFormats - /** - * Stores information about a stage to pass from the scheduler to SparkListeners. Also stores the - * metrics for all tasks that have completed, including redundant, speculated tasks. + * Stores information about a stage to pass from the scheduler to SparkListeners. Also + * stores the metrics for all tasks that have completed, including redundant, speculated tasks. */ +private[spark] class StageInfo( val stageId: Int, val name: String, @@ -36,70 +33,17 @@ class StageInfo( val numPartitions: Int, val numTasks: Int, val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = - mutable.Buffer[(TaskInfo, TaskMetrics)]() - ) extends JsonSerializable { + mutable.Buffer[(TaskInfo, TaskMetrics)]()) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None var emittedTaskSizeWarning = false - - override def toJson = { - val (taskInfoList, taskMetricsList) = taskInfos.toList.unzip - val taskInfoListJson = JArray(taskInfoList.map(_.toJson)) - val taskMetricsListJson = JArray(taskMetricsList.map(_.toJson)) - val submissionTimeJson = submissionTime.map(JInt(_)).getOrElse(JNothing) - val completionTimeJson = completionTime.map(JInt(_)).getOrElse(JNothing) - ("Stage ID" -> stageId) ~ - ("Stage Name" -> name) ~ - ("RDD Name" -> rddName) ~ - ("Number of Partitions" -> numPartitions) ~ - ("Number of Tasks" -> numTasks) ~ - ("Task Info List" -> taskInfoListJson) ~ - ("Task Metrics List" -> taskMetricsListJson) ~ - ("Submission Time" -> submissionTimeJson) ~ - ("Completion Time" -> completionTimeJson) ~ - ("Emitted Task Size Warning" -> emittedTaskSizeWarning) - } } +private[spark] object StageInfo { def fromStage(stage: Stage): StageInfo = { - new StageInfo( - stage.id, - stage.name, - stage.rdd.name, - stage.numPartitions, - stage.numTasks) - } - - def fromJson(json: JValue): StageInfo = { - implicit val format = DefaultFormats - val taskInfoListJson = (json \ "Task Info List").extract[List[JValue]] - val taskMetricsListJson = (json \ "Task Metrics List").extract[List[JValue]] - val taskInfo = taskInfoListJson.zip(taskMetricsListJson).map { case (info, metrics) => - (TaskInfo.fromJson(info), TaskMetrics.fromJson(metrics)) - }.toBuffer - - val stageInfo = new StageInfo( - (json \ "Stage ID").extract[Int], - (json \ "Stage Name").extract[String], - (json \ "RDD Name").extract[String], - (json \ "Number of Partitions").extract[Int], - (json \ "Number of Tasks").extract[Int], - taskInfo) - - stageInfo.submissionTime = - json \ "Submission Time" match { - case JNothing => None - case value: JValue => Some(value.extract[Long]) - } - stageInfo.completionTime = - json \ "Completion Time" match { - case JNothing => None - case value: JValue => Some(value.extract[Long]) - } - stageInfo.emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - stageInfo + new StageInfo(stage.id, stage.name, stage.rdd.name, stage.numPartitions, stage.numTasks) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 14d259ccac922..6183b125def99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,6 @@ package org.apache.spark.scheduler -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats - /** * Information about a running task attempt inside a TaskSet. */ @@ -31,8 +27,7 @@ class TaskInfo( val launchTime: Long, val executorId: String, val host: String, - val taskLocality: TaskLocality.TaskLocality) - extends JsonSerializable { + val taskLocality: TaskLocality.TaskLocality) { /** * The time when the task started remotely getting the result. Will not be set if the @@ -95,37 +90,4 @@ class TaskInfo( } def timeRunning(currentTime: Long): Long = currentTime - launchTime - - override def toJson = { - ("Task ID" -> taskId) ~ - ("Index" -> index) ~ - ("Launch Time" -> launchTime) ~ - ("Executor ID" -> executorId) ~ - ("Host" -> host) ~ - ("Locality" -> taskLocality.toString) ~ - ("Getting Result Time" -> gettingResultTime) ~ - ("Finish Time" -> finishTime) ~ - ("Failed" -> failed) ~ - ("Serialized Size" -> serializedSize) - } -} - -private[spark] -object TaskInfo { - def fromJson(json: JValue): TaskInfo = { - implicit val format = DefaultFormats - val taskInfo = new TaskInfo( - (json \ "Task ID").extract[Long], - (json \ "Index").extract[Int], - (json \ "Launch Time").extract[Long], - (json \ "Executor ID").extract[String], - (json \ "Host").extract[String], - TaskLocality.fromJson(json \ "Locality")) - - taskInfo.gettingResultTime = (json \ "Getting Result Time").extract[Long] - taskInfo.finishTime = (json \ "Finish Time").extract[Long] - taskInfo.failed = (json \ "Failed").extract[Boolean] - taskInfo.serializedSize = (json \ "Serialized Size").extract[Int] - taskInfo - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 766ce0cfd1e1b..308edb12edd5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,9 +17,6 @@ package org.apache.spark.scheduler -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats - private[spark] object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value @@ -29,9 +26,4 @@ private[spark] object TaskLocality extends Enumeration { def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { condition <= constraint } - - def fromJson(json: JValue): TaskLocality = { - implicit val format = DefaultFormats - TaskLocality.withName(json.extract[String]) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 16768bab62883..301d784b350a3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -19,13 +19,6 @@ package org.apache.spark.storage import java.util.UUID -import org.apache.spark.scheduler.JsonSerializable -import org.apache.spark.util.Utils - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats - /** * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different @@ -33,7 +26,7 @@ import net.liftweb.json.DefaultFormats * * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ -private[spark] sealed abstract class BlockId extends JsonSerializable { +private[spark] sealed abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -49,90 +42,42 @@ private[spark] sealed abstract class BlockId extends JsonSerializable { case o: BlockId => getClass == o.getClass && name.equals(o.name) case _ => false } - - override def toJson = "Type" -> Utils.getFormattedClassName(this) } private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex - - override def toJson = { - super.toJson ~ - ("RDD ID" -> rddId) ~ - ("Split Index" -> splitIndex) - } } private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - - override def toJson = { - super.toJson ~ - ("Shuffle ID" -> shuffleId) ~ - ("Map ID" -> mapId) ~ - ("Reduce ID" -> reduceId) - } } private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { def name = "broadcast_" + broadcastId - - override def toJson = { - super.toJson ~ - ("Broadcast ID" -> broadcastId) - } } private[spark] case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { def name = broadcastId.name + "_" + hType - - override def toJson = { - super.toJson ~ - ("Broadcast Block ID" -> broadcastId.toJson) ~ - ("Helper Type" -> hType) - } } private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { def name = "taskresult_" + taskId - - override def toJson = { - super.toJson ~ - ("Task ID" -> taskId) - } } private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId - - override def toJson = { - super.toJson ~ - ("Stream ID" -> streamId) ~ - ("Unique ID" -> uniqueId) - } } /** Id associated with temporary data managed as blocks. Not serializable. */ private[spark] case class TempBlockId(id: UUID) extends BlockId { def name = "temp_" + id - - override def toJson = { - val UUIDJson = Utils.UUIDToJson(id) - super.toJson ~ - ("Temp ID" -> UUIDJson) - } } // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id - - override def toJson = { - super.toJson ~ - ("Test ID" -> id) - } } private[spark] object BlockId { @@ -163,75 +108,4 @@ private[spark] object BlockId { case _ => throw new IllegalStateException("Unrecognized BlockId: " + id) } - - def fromJson(json: JValue): BlockId = { - implicit val format = DefaultFormats - val rddBlockId = Utils.getFormattedClassName(RDDBlockId) - val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) - val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) - val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) - val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) - val streamBlockId = Utils.getFormattedClassName(StreamBlockId) - val tempBlockId = Utils.getFormattedClassName(TempBlockId) - val testBlockId = Utils.getFormattedClassName(TestBlockId) - - (json \ "Type").extract[String] match { - case `rddBlockId` => rddBlockIdFromJson(json) - case `shuffleBlockId` => shuffleBlockIdFromJson(json) - case `broadcastBlockId` => broadcastBlockIdFromJson(json) - case `broadcastHelperBlockId` => broadcastHelperBlockIdFromJson(json) - case `taskResultBlockId` => taskResultBlockIdFromJson(json) - case `streamBlockId` => streamBlockIdFromJson(json) - case `tempBlockId` => tempBlockIdFromJson(json) - case `testBlockId` => testBlockIdFromJson(json) - } - } - - private def rddBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new RDDBlockId( - (json \ "RDD ID").extract[Int], - (json \ "Split Index").extract[Int]) - } - - private def shuffleBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new ShuffleBlockId( - (json \ "Shuffle ID").extract[Int], - (json \ "Map ID").extract[Int], - (json \ "Reduce ID").extract[Int]) - } - - private def broadcastBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new BroadcastBlockId((json \ "Broadcast ID").extract[Long]) - } - - private def broadcastHelperBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new BroadcastHelperBlockId( - broadcastBlockIdFromJson(json \ "Broadcast Block ID"), - (json \ "Helper Type").extract[String]) - } - - private def taskResultBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new TaskResultBlockId((json \ "Task ID").extract[Long]) - } - - private def streamBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new StreamBlockId( - (json \ "Stream ID").extract[Int], - (json \ "Unique ID").extract[Long]) - } - - private def tempBlockIdFromJson(json: JValue) = { - new TempBlockId(Utils.UUIDFromJson(json \ "Temp ID")) - } - - private def testBlockIdFromJson(json: JValue) = { - implicit val format = DefaultFormats - new TestBlockId((json \ "Test ID").extract[String]) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index ebfb3abac3039..87270c56a9cc4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,11 +20,6 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap import org.apache.spark.util.Utils -import org.apache.spark.scheduler.JsonSerializable - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats /** * This class represent an unique identifier for a BlockManager. @@ -39,7 +34,7 @@ private[spark] class BlockManagerId private ( private var host_ : String, private var port_ : Int, private var nettyPort_ : Int - ) extends Externalizable with JsonSerializable { + ) extends Externalizable { private def this() = this(null, null, 0, 0) // For deserialization only @@ -90,13 +85,6 @@ private[spark] class BlockManagerId private ( case _ => false } - - override def toJson = { - ("Executor ID" -> executorId_) ~ - ("Host" -> host_) ~ - ("Port" -> port_) ~ - ("Netty Port" -> nettyPort_) - } } @@ -126,13 +114,4 @@ private[spark] object BlockManagerId { blockManagerIdCache.putIfAbsent(id, id) blockManagerIdCache.get(id) } - - def fromJson(json: JValue): BlockManagerId = { - implicit val format = DefaultFormats - new BlockManagerId( - (json \ "Executor ID").extract[String], - (json \ "Host").extract[String], - (json \ "Port").extract[Int], - (json \ "Netty Port").extract[Int]) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 8465c4daacaac..893418fb8cad9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -30,11 +30,6 @@ import akka.pattern.ask import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.spark.scheduler.JsonSerializable - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -316,23 +311,6 @@ private[spark] object BlockManagerMasterActor { case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) - extends JsonSerializable { - override def toJson = { - ("Storage Level" -> storageLevel.toJson) ~ - ("Memory Size" -> memSize) ~ - ("Disk Size" -> diskSize) - } - } - - case object BlockStatus { - def fromJson(json: JValue): BlockStatus = { - implicit val format = DefaultFormats - new BlockStatus( - StorageLevel.fromJson(json \ "Storage Level"), - (json \ "Memory Size").extract[Long], - (json \ "Disk Size").extract[Long]) - } - } class BlockManagerInfo( val blockManagerId: BlockManagerId, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 51d084daee7da..1b7934d59fa1d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,12 +19,6 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} -import org.apache.spark.scheduler.JsonSerializable - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json.DefaultFormats - /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory @@ -38,7 +32,7 @@ class StorageLevel private( private var useMemory_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) - extends Externalizable with JsonSerializable { + extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. private def this(flags: Int, replication: Int) { @@ -111,13 +105,6 @@ class StorageLevel private( result += "%sx Replicated".format(replication) result } - - override def toJson = { - ("Use Disk" -> useDisk) ~ - ("Use Memory" -> useMemory) ~ - ("Deserialized" -> deserialized) ~ - ("Replication" -> replication) - } } @@ -160,13 +147,4 @@ object StorageLevel { storageLevelCache.putIfAbsent(level, level) storageLevelCache.get(level) } - - def fromJson(json: JValue): StorageLevel = { - implicit val format = DefaultFormats - new StorageLevel( - (json \ "Use Disk").extract[Boolean], - (json \ "Use Memory").extract[Boolean], - (json \ "Deserialized").extract[Boolean], - (json \ "Replication").extract[Int]) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 6931c13bd6976..69298abf5c2de 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -19,17 +19,12 @@ package org.apache.spark.storage import org.apache.spark.SparkContext import org.apache.spark.util.Utils -import org.apache.spark.scheduler.JsonSerializable - -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats import BlockManagerMasterActor.BlockStatus private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, - blocks: Map[BlockId, BlockStatus]) extends JsonSerializable { + blocks: Map[BlockId, BlockStatus]) { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) @@ -47,33 +42,6 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, case (rdd: RDDBlockId, status) => Some(rdd, status) case _ => None } - - override def toJson = { - val blocksJson = JArray( - blocks.toList.map { case (id, status) => - ("Block ID" -> id.toJson) ~ - ("Status" -> status.toJson) - }) - ("Block Manager ID" -> blockManagerId.toJson) ~ - ("Maximum Memory" -> maxMem) ~ - ("Blocks" -> blocksJson) - } -} - -private[spark] -case object StorageStatus { - def fromJson(json: JValue): StorageStatus = { - implicit val format = DefaultFormats - val blocks = (json \ "Blocks").extract[List[JValue]].map { block => - val id = BlockId.fromJson(block \ "Block ID") - val status = BlockStatus.fromJson(block \ "Status") - (id, status) - }.toMap - new StorageStatus( - BlockManagerId.fromJson(json \ "Block Manager ID"), - (json \ "Maximum Memory").extract[Long], - blocks) - } } case class RDDInfo( @@ -84,7 +52,7 @@ case class RDDInfo( numPartitions: Int, memSize: Long, diskSize: Long) - extends JsonSerializable with Ordered[RDDInfo] { + extends Ordered[RDDInfo] { override def toString = { ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, @@ -94,30 +62,6 @@ case class RDDInfo( override def compare(that: RDDInfo) = { this.id - that.id } - - override def toJson = { - ("RDD ID" -> id) ~ - ("Name" -> name) ~ - ("Storage Level" -> storageLevel.toJson) ~ - ("Number of Cached Partitions" -> numCachedPartitions) ~ - ("Number of Partitions" -> numPartitions) ~ - ("Memory Size" -> memSize) ~ - ("Disk Size" -> diskSize) - } -} - -case object RDDInfo { - def fromJson(json: JValue): RDDInfo = { - implicit val format = DefaultFormats - new RDDInfo( - (json \ "RDD ID").extract[Int], - (json \ "Name").extract[String], - StorageLevel.fromJson(json \ "Storage Level"), - (json \ "Number of Cached Partitions").extract[Int], - (json \ "Number of Partitions").extract[Int], - (json \ "Memory Size").extract[Long], - (json \ "Disk Size").extract[Long]) - } } /* Helper methods for storage-related objects */ 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 a67f962de6a60..2396462a2d0c6 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -31,6 +31,7 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils +import org.apache.spark.util.JsonProtocol import net.liftweb.json._ @@ -175,7 +176,7 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { val lines = Source.fromInputStream(bufferedStream).getLines() lines.foreach { line => currentLine = line - val event = SparkListenerEvent.fromJson(parse(line)) + val event = JsonProtocol.sparkEventFromJson(parse(line)) listenerBus.postToListeners(event, Seq(gatewayListener)) } } catch { diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 0cf0862ee325e..91e461c6fc7f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -23,6 +23,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.SparkContext import org.apache.spark.storage.StorageStatus import org.apache.spark.util.FileLogger +import org.apache.spark.util.JsonProtocol import net.liftweb.json.JsonAST._ @@ -55,7 +56,8 @@ private[spark] class GatewayUISparkListener(parent: SparkUI, live: Boolean) exte /** Log the event as JSON */ private def logEvent(event: SparkListenerEvent) { - logger.foreach(_.logLine(compactRender(event.toJson))) + val eventJson = JsonProtocol.sparkEventToJson(event) + logger.foreach(_.logLine(compactRender(eventJson))) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala new file mode 100644 index 0000000000000..d1e66e3ce43d9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -0,0 +1,725 @@ +/* + * 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.util + +import java.util.{UUID, Properties} + +import scala.collection.JavaConverters._ +import scala.collection.Map + +import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.scheduler._ +import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus +import org.apache.spark.storage._ +import org.apache.spark._ + +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats + + +private[spark] object JsonProtocol { + + /** + * JSON serialization methods for SparkListenerEvent's + */ + + def sparkEventToJson(event: SparkListenerEvent): JValue = { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + stageSubmittedEventToJson(stageSubmitted) + case stageCompleted: SparkListenerStageCompleted => + stageCompletedEventToJson(stageCompleted) + case taskStart: SparkListenerTaskStart => + taskStartEventToJson(taskStart) + case taskGettingResult: SparkListenerTaskGettingResult => + taskGettingResultEventToJson(taskGettingResult) + case taskEnd: SparkListenerTaskEnd => + taskEndEventToJson(taskEnd) + case jobStart: SparkListenerJobStart => + jobStartEventToJson(jobStart) + case jobEnd: SparkListenerJobEnd => + jobEndEventToJson(jobEnd) + case applicationStart: SparkListenerApplicationStart => + applicationStartEventToJson(applicationStart) + case storageStatusFetch: SparkListenerStorageStatusFetch => + storageStatusFetchEventToJson(storageStatusFetch) + case getRDDInfo: SparkListenerGetRDDInfo => + getRDDInfoEventToJson(getRDDInfo) + // SparkListenerShutdown is not supported + } + } + + def stageSubmittedEventToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { + val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) + val properties = propertiesToJson(stageSubmitted.properties) + ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ + ("Stage Info" -> stageInfo) ~ + ("Properties" -> properties) + } + + def stageCompletedEventToJson(stageCompleted: SparkListenerStageCompleted): JValue = { + val stageInfo = stageInfoToJson(stageCompleted.stageInfo) + ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ + ("Stage Info" -> stageInfo) + } + + def taskStartEventToJson(taskStart: SparkListenerTaskStart): JValue = { + val taskInfo = taskInfoToJson(taskStart.taskInfo) + ("Event" -> Utils.getFormattedClassName(taskStart)) ~ + ("Stage ID" -> taskStart.stageId) ~ + ("Task Info" -> taskInfo) + } + + def taskGettingResultEventToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { + val taskInfo = taskInfoToJson(taskGettingResult.taskInfo) + ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ + ("Task Info" -> taskInfo) + } + + def taskEndEventToJson(taskEnd: SparkListenerTaskEnd): JValue = { + val taskEndReason = taskEndReasonToJson(taskEnd.reason) + val taskInfo = taskInfoToJson(taskEnd.taskInfo) + val taskMetrics = taskMetricsToJson(taskEnd.taskMetrics) + ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ + ("Stage ID" -> taskEnd.stageId) ~ + ("Task Type" -> taskEnd.taskType) ~ + ("Task End Reason" -> taskEndReason) ~ + ("Task Info" -> taskInfo) ~ + ("Task Metrics" -> taskMetrics) + } + + def jobStartEventToJson(jobStart: SparkListenerJobStart): JValue = { + val properties = propertiesToJson(jobStart.properties) + ("Event" -> Utils.getFormattedClassName(jobStart)) ~ + ("Job ID" -> jobStart.jobId) ~ + ("Stage IDs" -> jobStart.stageIds) ~ + ("Properties" -> properties) + } + + def jobEndEventToJson(jobEnd: SparkListenerJobEnd): JValue = { + val jobResult = jobResultToJson(jobEnd.jobResult) + ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ + ("Job ID" -> jobEnd.jobId) ~ + ("Job Result" -> jobResult) + } + + def applicationStartEventToJson(applicationStart: SparkListenerApplicationStart): JValue = { + val environmentDetails = applicationStart.environmentDetails + val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) + val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) + val systemProperties = mapToJson(environmentDetails("System Properties").toMap) + val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("JVM Information" -> jvmInformation) ~ + ("Spark Properties" -> sparkProperties) ~ + ("System Properties" -> systemProperties) ~ + ("Classpath Entries" -> classpathEntries) + } + + def storageStatusFetchEventToJson(storageStatusFetch: SparkListenerStorageStatusFetch): JValue = { + val storageStatusList = + JArray(storageStatusFetch.storageStatusList.map(storageStatusToJson).toList) + ("Event" -> Utils.getFormattedClassName(storageStatusFetch)) ~ + ("Storage Status List" -> storageStatusList) + } + + def getRDDInfoEventToJson(getRDDInfo: SparkListenerGetRDDInfo): JValue = { + val rddInfoList = JArray(getRDDInfo.rddInfoList.map(rddInfoToJson).toList) + ("Event" -> Utils.getFormattedClassName(getRDDInfo)) ~ + ("RDD Info List" -> rddInfoList) + } + + /** + * JSON serialization methods for classes SparkListenerEvent's depend on + */ + + def stageInfoToJson(stageInfo: StageInfo): JValue = { + val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => + ("Task Info" -> taskInfoToJson(info)) ~ + ("Task Metrics" -> taskMetricsToJson(metrics)) + }.toList) + val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) + val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + ("Stage ID" -> stageInfo.stageId) ~ + ("Stage Name" -> stageInfo.name) ~ + ("RDD Name" -> stageInfo.rddName) ~ + ("Number of Partitions" -> stageInfo.numPartitions) ~ + ("Number of Tasks" -> stageInfo.numTasks) ~ + ("Task Infos" -> taskInfos) ~ + ("Submission Time" -> submissionTime) ~ + ("Completion Time" -> completionTime) ~ + ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) + } + + def taskInfoToJson(taskInfo: TaskInfo): JValue = { + ("Task ID" -> taskInfo.taskId) ~ + ("Index" -> taskInfo.index) ~ + ("Launch Time" -> taskInfo.launchTime) ~ + ("Executor ID" -> taskInfo.executorId) ~ + ("Host" -> taskInfo.host) ~ + ("Locality" -> taskInfo.taskLocality.toString) ~ + ("Getting Result Time" -> taskInfo.gettingResultTime) ~ + ("Finish Time" -> taskInfo.finishTime) ~ + ("Failed" -> taskInfo.failed) ~ + ("Serialized Size" -> taskInfo.serializedSize) + } + + def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { + val shuffleReadMetrics = + taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) + val shuffleWriteMetrics = + taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) + ("Host Name" -> taskMetrics.hostname) ~ + ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ + ("Executor Run Time" -> taskMetrics.executorRunTime) ~ + ("Result Size" -> taskMetrics.resultSize) ~ + ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ + ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ + ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ + ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ + ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ + ("Shuffle Write Metrics" -> shuffleWriteMetrics) + } + + def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { + ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ + ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ + ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ + ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ + ("Remote Fetch Time" -> shuffleReadMetrics.remoteFetchTime) ~ + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + } + + def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { + ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + } + + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { + val reason = Utils.getFormattedClassName(taskEndReason) + val json = taskEndReason match { + case fetchFailed: FetchFailed => + val blockManagerAddress = blockManagerIdToJson(fetchFailed.bmAddress) + ("Block Manager Address" -> blockManagerAddress) ~ + ("Shuffle ID" -> fetchFailed.shuffleId) ~ + ("Map ID" -> fetchFailed.mapId) ~ + ("Reduce ID" -> fetchFailed.reduceId) + case exceptionFailure: ExceptionFailure => + val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) + val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) + ("Class Name" -> exceptionFailure.className) ~ + ("Description" -> exceptionFailure.description) ~ + ("Stack Trace" -> stackTrace) ~ + ("Metrics" -> metrics) + case _ => Utils.emptyJson + } + ("Reason" -> reason) ~ json + } + + def blockManagerIdToJson(blockManagerId: BlockManagerId): JValue = { + ("Executor ID" -> blockManagerId.executorId) ~ + ("Host" -> blockManagerId.host) ~ + ("Port" -> blockManagerId.port) ~ + ("Netty Port" -> blockManagerId.nettyPort) + } + + def jobResultToJson(jobResult: JobResult): JValue = { + val result = Utils.getFormattedClassName(jobResult) + val json = jobResult match { + case JobSucceeded => Utils.emptyJson + case jobFailed: JobFailed => + val exception = exceptionToJson(jobFailed.exception) + ("Exception" -> exception) ~ + ("Failed Stage ID" -> jobFailed.failedStageId) + } + ("Result" -> result) ~ json + } + + def storageStatusToJson(storageStatus: StorageStatus): JValue = { + val blockManagerId = blockManagerIdToJson(storageStatus.blockManagerId) + val blocks = JArray( + storageStatus.blocks.toList.map { case (id, status) => + ("Block ID" -> blockIdToJson(id)) ~ + ("Status" -> blockStatusToJson(status)) + }) + ("Block Manager ID" -> blockManagerId) ~ + ("Maximum Memory" -> storageStatus.maxMem) ~ + ("Blocks" -> blocks) + } + + def rddInfoToJson(rddInfo: RDDInfo): JValue = { + val storageLevel = storageLevelToJson(rddInfo.storageLevel) + ("RDD ID" -> rddInfo.id) ~ + ("Name" -> rddInfo.name) ~ + ("Storage Level" -> storageLevel) ~ + ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ + ("Number of Partitions" -> rddInfo.numPartitions) ~ + ("Memory Size" -> rddInfo.memSize) ~ + ("Disk Size" -> rddInfo.diskSize) + } + + def storageLevelToJson(storageLevel: StorageLevel): JValue = { + ("Use Disk" -> storageLevel.useDisk) ~ + ("Use Memory" -> storageLevel.useMemory) ~ + ("Deserialized" -> storageLevel.deserialized) ~ + ("Replication" -> storageLevel.replication) + } + + def blockIdToJson(blockId: BlockId): JValue = { + val blockType = Utils.getFormattedClassName(blockId) + val json: JObject = blockId match { + case rddBlockId: RDDBlockId => + ("RDD ID" -> rddBlockId.rddId) ~ + ("Split Index" -> rddBlockId.splitIndex) + case shuffleBlockId: ShuffleBlockId => + ("Shuffle ID" -> shuffleBlockId.shuffleId) ~ + ("Map ID" -> shuffleBlockId.mapId) ~ + ("Reduce ID" -> shuffleBlockId.reduceId) + case broadcastBlockId: BroadcastBlockId => + "Broadcast ID" -> broadcastBlockId.broadcastId + case broadcastHelperBlockId: BroadcastHelperBlockId => + ("Broadcast Block ID" -> blockIdToJson(broadcastHelperBlockId.broadcastId)) ~ + ("Helper Type" -> broadcastHelperBlockId.hType) + case taskResultBlockId: TaskResultBlockId => + "Task ID" -> taskResultBlockId.taskId + case streamBlockId: StreamBlockId => + ("Stream ID" -> streamBlockId.streamId) ~ + ("Unique ID" -> streamBlockId.uniqueId) + case tempBlockId: TempBlockId => + val uuid = UUIDToJson(tempBlockId.id) + "Temp ID" -> uuid + case testBlockId: TestBlockId => + "Test ID" -> testBlockId.id + } + ("Type" -> blockType) ~ json + } + + def blockStatusToJson(blockStatus: BlockStatus): JValue = { + val storageLevel = storageLevelToJson(blockStatus.storageLevel) + ("Storage Level" -> storageLevel) ~ + ("Memory Size" -> blockStatus.memSize) ~ + ("Disk Size" -> blockStatus.diskSize) + } + + /** + * Util JSON serialization methods + */ + + def mapToJson(m: Map[String, String]): JValue = { + val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } + JObject(jsonFields.toList) + } + + def propertiesToJson(properties: Properties): JValue = { + Option(properties).map { p => + mapToJson(p.asScala) + }.getOrElse(JNothing) + } + + def UUIDToJson(id: UUID): JValue = { + ("Least Significant Bits" -> id.getLeastSignificantBits) ~ + ("Most Significant Bits" -> id.getMostSignificantBits) + } + + def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { + JArray(stackTrace.map { case line => + ("Declaring Class" -> line.getClassName) ~ + ("Method Name" -> line.getMethodName) ~ + ("File Name" -> line.getFileName) ~ + ("Line Number" -> line.getLineNumber) + }.toList) + } + + def exceptionToJson(exception: Exception): JValue = { + ("Message" -> exception.toString) ~ + ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) + } + + /** + * JSON deserialization methods for SparkListenerEvent's + */ + + def sparkEventFromJson(json: JValue): SparkListenerEvent = { + val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) + val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) + val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) + val taskGettingResult = Utils.getFormattedClassName(SparkListenerTaskGettingResult) + val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) + val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) + val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) + val getRDDInfo = Utils.getFormattedClassName(SparkListenerGetRDDInfo) + // SparkListenerShutdown is not supported + + implicit val format = DefaultFormats + (json \ "Event").extract[String] match { + case `stageSubmitted` => stageSubmittedFromJson(json) + case `stageCompleted` => stageCompletedFromJson(json) + case `taskStart` => taskStartFromJson(json) + case `taskGettingResult` => taskGettingResultFromJson(json) + case `taskEnd` => taskEndFromJson(json) + case `jobStart` => jobStartFromJson(json) + case `jobEnd` => jobEndFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `storageStatusFetch` => storageStatusFetchFromJson(json) + case `getRDDInfo` => getRDDInfoFromJson(json) + } + } + + def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + val properties = propertiesFromJson(json \ "Properties") + new SparkListenerStageSubmitted(stageInfo, properties) + } + + def stageCompletedFromJson(json: JValue): SparkListenerStageCompleted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + new SparkListenerStageCompleted(stageInfo) + } + + def taskStartFromJson(json: JValue): SparkListenerTaskStart = { + implicit val format = DefaultFormats + val stageId = (json \ "Stage ID").extract[Int] + val taskInfo = taskInfoFromJson(json \ "Task Info") + new SparkListenerTaskStart(stageId, taskInfo) + } + + def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { + val taskInfo = taskInfoFromJson(json \ "Task Info") + new SparkListenerTaskGettingResult(taskInfo) + } + + def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { + implicit val format = DefaultFormats + val stageId = (json \ "Stage ID").extract[Int] + val taskType = (json \ "Task Type").extract[String] + val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") + val taskInfo = taskInfoFromJson(json \ "Task Info") + val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") + new SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + } + + def jobStartFromJson(json: JValue): SparkListenerJobStart = { + implicit val format = DefaultFormats + val jobId = (json \ "Job ID").extract[Int] + val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) + val properties = propertiesFromJson(json \ "Properties") + new SparkListenerJobStart(jobId, stageIds, properties) + } + + def jobEndFromJson(json: JValue): SparkListenerJobEnd = { + implicit val format = DefaultFormats + val jobId = (json \ "Job ID").extract[Int] + val jobResult = jobResultFromJson(json \ "Job Result") + new SparkListenerJobEnd(jobId, jobResult) + } + + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val environmentDetails = Map[String, Seq[(String, String)]]( + "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, + "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "System Properties" -> mapFromJson(json \ "System Properties").toSeq, + "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) + new SparkListenerApplicationStart(environmentDetails) + } + + def storageStatusFetchFromJson(json: JValue): SparkListenerStorageStatusFetch = { + implicit val format = DefaultFormats + val storageStatusList = + (json \ "Storage Status List").extract[List[JValue]].map(storageStatusFromJson) + new SparkListenerStorageStatusFetch(storageStatusList) + } + + def getRDDInfoFromJson(json: JValue): SparkListenerGetRDDInfo = { + implicit val format = DefaultFormats + val rddInfoList = (json \ "RDD Info List").extract[List[JValue]].map(rddInfoFromJson) + new SparkListenerGetRDDInfo(rddInfoList) + } + + /** + * JSON deserialization methods for classes SparkListenerEvent's depend on + */ + + def stageInfoFromJson(json: JValue): StageInfo = { + implicit val format = DefaultFormats + val stageId = (json \ "Stage ID").extract[Int] + val stageName = (json \ "Stage Name").extract[String] + val rddName = (json \ "RDD Name").extract[String] + val numPartitions = (json \ "Number of Partitions").extract[Int] + val numTasks = (json \ "Number of Tasks").extract[Int] + val taskInfos = (json \ "Task Infos").extract[List[JValue]].map { value => + (taskInfoFromJson(value \ "Task Info"), taskMetricsFromJson(value \ "Task Metrics")) + }.toBuffer + val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) + val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] + + val stageInfo = new StageInfo(stageId, stageName, rddName, numPartitions, numTasks, taskInfos) + stageInfo.submissionTime = submissionTime + stageInfo.completionTime = completionTime + stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning + stageInfo + } + + def taskInfoFromJson(json: JValue): TaskInfo = { + implicit val format = DefaultFormats + val taskId = (json \ "Task ID").extract[Long] + val index = (json \ "Index").extract[Int] + val launchTime = (json \ "Launch Time").extract[Long] + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val taskLocality = TaskLocality.withName((json \ "Locality").extract[String]) + val gettingResultTime = (json \ "Getting Result Time").extract[Long] + val finishTime = (json \ "Finish Time").extract[Long] + val failed = (json \ "Failed").extract[Boolean] + val serializedSize = (json \ "Serialized Size").extract[Int] + + val taskInfo = new TaskInfo(taskId, index, launchTime, executorId, host, taskLocality) + taskInfo.gettingResultTime = gettingResultTime + taskInfo.finishTime = finishTime + taskInfo.failed = failed + taskInfo.serializedSize = serializedSize + taskInfo + } + + def taskMetricsFromJson(json: JValue): TaskMetrics = { + implicit val format = DefaultFormats + val metrics = new TaskMetrics + metrics.hostname = (json \ "Host Name").extract[String] + metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] + metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] + metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] + metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] + metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] + metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] + metrics.shuffleReadMetrics = + Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) + metrics.shuffleWriteMetrics = + Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + metrics + } + + def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { + implicit val format = DefaultFormats + val metrics = new ShuffleReadMetrics + metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] + metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] + metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] + metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] + metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] + metrics.remoteFetchTime = (json \ "Remote Fetch Time").extract[Long] + metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] + metrics + } + + def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { + implicit val format = DefaultFormats + val metrics = new ShuffleWriteMetrics + metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] + metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] + metrics + } + + def taskEndReasonFromJson(json: JValue): TaskEndReason = { + implicit val format = DefaultFormats + val success = Utils.getFormattedClassName(Success) + val resubmitted = Utils.getFormattedClassName(Resubmitted) + val fetchFailed = Utils.getFormattedClassName(FetchFailed) + val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) + val taskResultLost = Utils.getFormattedClassName(TaskResultLost) + val taskKilled = Utils.getFormattedClassName(TaskKilled) + val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) + val unknownReason = Utils.getFormattedClassName(UnknownReason) + + (json \ "Reason").extract[String] match { + case `success` => Success + case `resubmitted` => Resubmitted + case `fetchFailed` => + val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId) + case `exceptionFailure` => + val className = (json \ "Class Name").extract[String] + val description = (json \ "Description").extract[String] + val stackTrace = stackTraceFromJson(json \ "Stack Trace") + val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) + new ExceptionFailure(className, description, stackTrace, metrics) + case `taskResultLost` => TaskResultLost + case `taskKilled` => TaskKilled + case `executorLostFailure` => ExecutorLostFailure + case `unknownReason` => UnknownReason + } + } + + def blockManagerIdFromJson(json: JValue): BlockManagerId = { + implicit val format = DefaultFormats + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val port = (json \ "Port").extract[Int] + val nettyPort = (json \ "Netty Port").extract[Int] + BlockManagerId(executorId, host, port, nettyPort) + } + + def jobResultFromJson(json: JValue): JobResult = { + implicit val format = DefaultFormats + val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) + val jobFailed = Utils.getFormattedClassName(JobFailed) + + (json \ "Result").extract[String] match { + case `jobSucceeded` => JobSucceeded + case `jobFailed` => + val exception = exceptionFromJson(json \ "Exception") + val failedStageId = (json \ "Failed Stage ID").extract[Int] + new JobFailed(exception, failedStageId) + } + } + + def storageStatusFromJson(json: JValue): StorageStatus = { + implicit val format = DefaultFormats + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + val maxMem = (json \ "Maximum Memory").extract[Long] + val blocks = (json \ "Blocks").extract[List[JValue]].map { block => + val id = blockIdFromJson(block \ "Block ID") + val status = blockStatusFromJson(block \ "Status") + (id, status) + }.toMap + new StorageStatus(blockManagerId, maxMem, blocks) + } + + def rddInfoFromJson(json: JValue): RDDInfo = { + implicit val format = DefaultFormats + val rddId = (json \ "RDD ID").extract[Int] + val name = (json \ "Name").extract[String] + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] + val numPartitions = (json \ "Number of Partitions").extract[Int] + val memSize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + new RDDInfo(rddId, name, storageLevel, numCachedPartitions, numPartitions, memSize, diskSize) + } + + def storageLevelFromJson(json: JValue): StorageLevel = { + implicit val format = DefaultFormats + val useDisk = (json \ "Use Disk").extract[Boolean] + val useMemory = (json \ "Use Memory").extract[Boolean] + val deserialized = (json \ "Deserialized").extract[Boolean] + val replication = (json \ "Replication").extract[Int] + StorageLevel(useDisk, useMemory, deserialized, replication) + } + + def blockIdFromJson(json: JValue): BlockId = { + implicit val format = DefaultFormats + val rddBlockId = Utils.getFormattedClassName(RDDBlockId) + val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) + val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) + val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) + val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) + val streamBlockId = Utils.getFormattedClassName(StreamBlockId) + val tempBlockId = Utils.getFormattedClassName(TempBlockId) + val testBlockId = Utils.getFormattedClassName(TestBlockId) + + (json \ "Type").extract[String] match { + case `rddBlockId` => + val rddId = (json \ "RDD ID").extract[Int] + val splitIndex = (json \ "Split Index").extract[Int] + new RDDBlockId(rddId, splitIndex) + case `shuffleBlockId` => + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new ShuffleBlockId(shuffleId, mapId, reduceId) + case `broadcastBlockId` => + val broadcastId = (json \ "Broadcast ID").extract[Long] + new BroadcastBlockId(broadcastId) + case `broadcastHelperBlockId` => + val broadcastBlockId = + blockIdFromJson(json \ "Broadcast Block ID").asInstanceOf[BroadcastBlockId] + val hType = (json \ "Helper Type").extract[String] + new BroadcastHelperBlockId(broadcastBlockId, hType) + case `taskResultBlockId` => + val taskId = (json \ "Task ID").extract[Long] + new TaskResultBlockId(taskId) + case `streamBlockId` => + val streamId = (json \ "Stream ID").extract[Int] + val uniqueId = (json \ "Unique ID").extract[Long] + new StreamBlockId(streamId, uniqueId) + case `tempBlockId` => + val tempId = UUIDFromJson(json \ "Temp ID") + new TempBlockId(tempId) + case `testBlockId` => + val testId = (json \ "Test ID").extract[String] + new TestBlockId(testId) + } + } + + def blockStatusFromJson(json: JValue): BlockStatus = { + implicit val format = DefaultFormats + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val memorySize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + new BlockStatus(storageLevel, memorySize, diskSize) + } + + /** + * Util JSON deserialization methods + */ + + def mapFromJson(json: JValue): Map[String, String] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap + } + + def propertiesFromJson(json: JValue): Properties = { + val properties = new Properties() + if (json != JNothing) { + mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } + } + properties + } + + def UUIDFromJson(json: JValue): UUID = { + implicit val format = DefaultFormats + new UUID( + (json \ "Least Significant Bits").extract[Long], + (json \ "Most Significant Bits").extract[Long]) + } + + def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { + implicit val format = DefaultFormats + json.extract[List[JValue]].map { line => + new StackTraceElement( + (line \ "Declaring Class").extract[String], + (line \ "Method Name").extract[String], + (line \ "File Name").extract[String], + (line \ "Line Number").extract[Int]) + }.toArray + } + + def exceptionFromJson(json: JValue): Exception = { + implicit val format = DefaultFormats + val e = new Exception((json \ "Message").extract[String]) + e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) + e + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c8df917c01558..bfb8cd2110930 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -20,12 +20,10 @@ package org.apache.spark.util import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} import java.nio.ByteBuffer -import java.util.{Properties, Locale, Random, UUID} +import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} -import java.lang.StackTraceElement import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source @@ -41,8 +39,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.{SparkConf, SparkException, Logging} import net.liftweb.json.JsonAST._ -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.DefaultFormats /** * Various utility methods used by Spark. @@ -877,81 +873,12 @@ private[spark] object Utils extends Logging { obj.getClass.getSimpleName.replace("$", "") } - /** Convert a (String, String) map to a JSON object */ - def mapToJson(m: Map[String, String]): JValue = { - val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } - JObject(jsonFields.toList) - } - - /** Convert a JSON object to a (String, String) map */ - def mapFromJson(json: JValue): Map[String, String] = { - val jsonFields = json.asInstanceOf[JObject].obj - jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap - } - - /** Convert a java Properties to a JSON object */ - def propertiesToJson(properties: Properties): JValue = { - Option(properties).map { p => - mapToJson(p.asScala) - }.getOrElse(JNothing) - } - - /** Convert a JSON object to a java Properties */ - def propertiesFromJson(json: JValue): Properties = { - val properties = new Properties() - if (json != JNothing) { - mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } + def jsonOption(json: JValue): Option[JValue] = { + json match { + case JNothing => None + case value: JValue => Some(value) } - properties - } - - /** Convert a java UUID to a JSON object */ - def UUIDToJson(id: UUID): JValue = { - ("Least Significant Bits" -> id.getLeastSignificantBits) ~ - ("Most Significant Bits" -> id.getMostSignificantBits) } - /** Convert a JSON object to a java UUID */ - def UUIDFromJson(json: JValue): UUID = { - implicit val format = DefaultFormats - new UUID( - (json \ "Least Significant Bits").extract[Long], - (json \ "Most Significant Bits").extract[Long]) - } - - /** Convert a java stack trace to a JSON object */ - def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { - JArray(stackTrace.map { case line => - ("Declaring Class" -> line.getClassName) ~ - ("Method Name" -> line.getMethodName) ~ - ("File Name" -> line.getFileName) ~ - ("Line Number" -> line.getLineNumber) - }.toList) - } - - /** Convert a JSON object to a java stack trace */ - def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { - implicit val format = DefaultFormats - json.extract[List[JValue]].map { line => - new StackTraceElement( - (line \ "Declaring Class").extract[String], - (line \ "Method Name").extract[String], - (line \ "File Name").extract[String], - (line \ "Line Number").extract[Int]) - }.toArray - } - - /** Convert an Exception to a JSON object */ - def exceptionToJson(exception: Exception): JValue = { - ("Message" -> exception.toString) ~ - ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) - } - - /** Convert a JSON object to an Exception */ - def exceptionFromJson(json: JValue): Exception = { - implicit val format = DefaultFormats - val e = new Exception((json \ "Message").extract[String]) - e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) - e - } + def emptyJson = JObject(List[JField]()) } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 6445db0063fc4..b46d853ef9e97 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{ExecutorRunner, DriverRunner} +import org.apache.spark.deploy.{ExecutorState, DriverDescription, Command, ApplicationDescription} class JsonProtocolSuite extends FunSuite { From 70e7e7acf09d8efd2c7e459ee450c1db140b8f5a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 21 Feb 2014 18:56:26 -0800 Subject: [PATCH 27/68] Formatting changes --- .../src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- .../org/apache/spark/deploy/master/ui/IndexPage.scala | 5 ++--- .../org/apache/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../scala/org/apache/spark/executor/TaskMetrics.scala | 1 + .../scala/org/apache/spark/storage/StorageUtils.scala | 10 ++-------- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/deploy/JsonProtocolSuite.scala | 1 - 8 files changed, 11 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 148958542cece..d024bcddb21f4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -215,6 +215,7 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { + val env = SparkEnv.get val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 690fc4654a054..f29a6ad2e7b92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -24,11 +24,11 @@ import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils -import org.apache.spark.util.{Utils} -import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 6b1e1a77af290..13a5223014f1a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -18,18 +18,17 @@ package org.apache.spark.deploy.master.ui import scala.concurrent.Await -import scala.concurrent.duration._ import scala.xml.Node import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.{JsonProtocol, DeployWebUI} +import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.ui.UIUtils -import org.apache.spark.util.{Utils} +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 602e38e8733f2..3089acffb8d98 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -24,12 +24,12 @@ import akka.pattern.ask import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.UIUtils -import org.apache.spark.util.{Utils} -import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: WorkerWebUI) { val workerActor = parent.worker.self diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 4aa51d0426a99..bf421da04d6fe 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -74,6 +74,7 @@ object TaskMetrics { private[spark] def empty(): TaskMetrics = new TaskMetrics } + class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 69298abf5c2de..0126229686aa2 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -44,14 +44,8 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } } -case class RDDInfo( - id: Int, - name: String, - storageLevel: StorageLevel, - numCachedPartitions: Int, - numPartitions: Int, - memSize: Long, - diskSize: Long) +case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, + numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) extends Ordered[RDDInfo] { override def toString = { ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index e38bbc091d128..2f4a3fd52f6cf 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -248,4 +248,4 @@ private[spark] case class TaskUIData( private[spark] object JobProgressListener { val DEFAULT_RETAINED_STAGES = 1000 val DEFAULT_POOL_NAME = "default" -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index b46d853ef9e97..6445db0063fc4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,6 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{ExecutorRunner, DriverRunner} -import org.apache.spark.deploy.{ExecutorState, DriverDescription, Command, ApplicationDescription} class JsonProtocolSuite extends FunSuite { From 6631c02a8791d0321f003bb339344445f4dd0cab Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 24 Feb 2014 10:52:21 -0800 Subject: [PATCH 28/68] More formatting changes, this time mainly for Json DSL --- .../scala/org/apache/spark/CacheManager.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 1 - .../spark/storage/BlockManagerWorker.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 1 - .../org/apache/spark/util/JsonProtocol.scala | 195 +++++++++--------- 5 files changed, 99 insertions(+), 102 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index b38af2497d3d9..bca74825efd28 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -18,7 +18,7 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} -import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, RDDBlockId} +import org.apache.spark.storage.{BlockManager, StorageLevel, RDDBlockId} import org.apache.spark.rdd.RDD diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 780a3a15dd15b..47ea144c0c799 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -25,7 +25,6 @@ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} import scala.concurrent.{Await, Future} -import scala.concurrent.duration.Duration import scala.concurrent.duration._ import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index 3efe738a08f66..c7766a3a65671 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.nio.ByteBuffer -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.network._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index eb5a18521683e..82089b923d190 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -18,7 +18,6 @@ package org.apache.spark.storage import java.util.LinkedHashMap -import java.util.concurrent.ArrayBlockingQueue import java.nio.ByteBuffer import collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} 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 d1e66e3ce43d9..f3af976cf1098 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,7 +32,6 @@ import net.liftweb.json.JsonDSL._ import net.liftweb.json.JsonAST._ import net.liftweb.json.DefaultFormats - private[spark] object JsonProtocol { /** @@ -69,27 +68,27 @@ private[spark] object JsonProtocol { val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) val properties = propertiesToJson(stageSubmitted.properties) ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ - ("Stage Info" -> stageInfo) ~ - ("Properties" -> properties) + ("Stage Info" -> stageInfo) ~ + ("Properties" -> properties) } def stageCompletedEventToJson(stageCompleted: SparkListenerStageCompleted): JValue = { val stageInfo = stageInfoToJson(stageCompleted.stageInfo) ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ - ("Stage Info" -> stageInfo) + ("Stage Info" -> stageInfo) } def taskStartEventToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskInfoToJson(taskStart.taskInfo) ("Event" -> Utils.getFormattedClassName(taskStart)) ~ - ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfo) + ("Stage ID" -> taskStart.stageId) ~ + ("Task Info" -> taskInfo) } def taskGettingResultEventToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskInfoToJson(taskGettingResult.taskInfo) ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfo) + ("Task Info" -> taskInfo) } def taskEndEventToJson(taskEnd: SparkListenerTaskEnd): JValue = { @@ -97,26 +96,26 @@ private[spark] object JsonProtocol { val taskInfo = taskInfoToJson(taskEnd.taskInfo) val taskMetrics = taskMetricsToJson(taskEnd.taskMetrics) ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ - ("Stage ID" -> taskEnd.stageId) ~ - ("Task Type" -> taskEnd.taskType) ~ - ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfo) ~ - ("Task Metrics" -> taskMetrics) + ("Stage ID" -> taskEnd.stageId) ~ + ("Task Type" -> taskEnd.taskType) ~ + ("Task End Reason" -> taskEndReason) ~ + ("Task Info" -> taskInfo) ~ + ("Task Metrics" -> taskMetrics) } def jobStartEventToJson(jobStart: SparkListenerJobStart): JValue = { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ - ("Job ID" -> jobStart.jobId) ~ - ("Stage IDs" -> jobStart.stageIds) ~ - ("Properties" -> properties) + ("Job ID" -> jobStart.jobId) ~ + ("Stage IDs" -> jobStart.stageIds) ~ + ("Properties" -> properties) } def jobEndEventToJson(jobEnd: SparkListenerJobEnd): JValue = { val jobResult = jobResultToJson(jobEnd.jobResult) ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ - ("Job ID" -> jobEnd.jobId) ~ - ("Job Result" -> jobResult) + ("Job ID" -> jobEnd.jobId) ~ + ("Job Result" -> jobResult) } def applicationStartEventToJson(applicationStart: SparkListenerApplicationStart): JValue = { @@ -126,10 +125,10 @@ private[spark] object JsonProtocol { val systemProperties = mapToJson(environmentDetails("System Properties").toMap) val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ - ("JVM Information" -> jvmInformation) ~ - ("Spark Properties" -> sparkProperties) ~ - ("System Properties" -> systemProperties) ~ - ("Classpath Entries" -> classpathEntries) + ("JVM Information" -> jvmInformation) ~ + ("Spark Properties" -> sparkProperties) ~ + ("System Properties" -> systemProperties) ~ + ("Classpath Entries" -> classpathEntries) } def storageStatusFetchEventToJson(storageStatusFetch: SparkListenerStorageStatusFetch): JValue = { @@ -142,7 +141,7 @@ private[spark] object JsonProtocol { def getRDDInfoEventToJson(getRDDInfo: SparkListenerGetRDDInfo): JValue = { val rddInfoList = JArray(getRDDInfo.rddInfoList.map(rddInfoToJson).toList) ("Event" -> Utils.getFormattedClassName(getRDDInfo)) ~ - ("RDD Info List" -> rddInfoList) + ("RDD Info List" -> rddInfoList) } /** @@ -152,32 +151,32 @@ private[spark] object JsonProtocol { def stageInfoToJson(stageInfo: StageInfo): JValue = { val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => ("Task Info" -> taskInfoToJson(info)) ~ - ("Task Metrics" -> taskMetricsToJson(metrics)) + ("Task Metrics" -> taskMetricsToJson(metrics)) }.toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ - ("Stage Name" -> stageInfo.name) ~ - ("RDD Name" -> stageInfo.rddName) ~ - ("Number of Partitions" -> stageInfo.numPartitions) ~ - ("Number of Tasks" -> stageInfo.numTasks) ~ - ("Task Infos" -> taskInfos) ~ - ("Submission Time" -> submissionTime) ~ - ("Completion Time" -> completionTime) ~ - ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) + ("Stage Name" -> stageInfo.name) ~ + ("RDD Name" -> stageInfo.rddName) ~ + ("Number of Partitions" -> stageInfo.numPartitions) ~ + ("Number of Tasks" -> stageInfo.numTasks) ~ + ("Task Infos" -> taskInfos) ~ + ("Submission Time" -> submissionTime) ~ + ("Completion Time" -> completionTime) ~ + ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) } def taskInfoToJson(taskInfo: TaskInfo): JValue = { ("Task ID" -> taskInfo.taskId) ~ - ("Index" -> taskInfo.index) ~ - ("Launch Time" -> taskInfo.launchTime) ~ - ("Executor ID" -> taskInfo.executorId) ~ - ("Host" -> taskInfo.host) ~ - ("Locality" -> taskInfo.taskLocality.toString) ~ - ("Getting Result Time" -> taskInfo.gettingResultTime) ~ - ("Finish Time" -> taskInfo.finishTime) ~ - ("Failed" -> taskInfo.failed) ~ - ("Serialized Size" -> taskInfo.serializedSize) + ("Index" -> taskInfo.index) ~ + ("Launch Time" -> taskInfo.launchTime) ~ + ("Executor ID" -> taskInfo.executorId) ~ + ("Host" -> taskInfo.host) ~ + ("Locality" -> taskInfo.taskLocality.toString) ~ + ("Getting Result Time" -> taskInfo.gettingResultTime) ~ + ("Finish Time" -> taskInfo.finishTime) ~ + ("Failed" -> taskInfo.failed) ~ + ("Serialized Size" -> taskInfo.serializedSize) } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { @@ -186,30 +185,30 @@ private[spark] object JsonProtocol { val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) ("Host Name" -> taskMetrics.hostname) ~ - ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ - ("Executor Run Time" -> taskMetrics.executorRunTime) ~ - ("Result Size" -> taskMetrics.resultSize) ~ - ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ - ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ - ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ - ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ - ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ - ("Shuffle Write Metrics" -> shuffleWriteMetrics) + ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ + ("Executor Run Time" -> taskMetrics.executorRunTime) ~ + ("Result Size" -> taskMetrics.resultSize) ~ + ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ + ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ + ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ + ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ + ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ + ("Shuffle Write Metrics" -> shuffleWriteMetrics) } def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ - ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ - ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ - ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ - ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ - ("Remote Fetch Time" -> shuffleReadMetrics.remoteFetchTime) ~ - ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ + ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ + ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ + ("Remote Fetch Time" -> shuffleReadMetrics.remoteFetchTime) ~ + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) } def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -218,16 +217,16 @@ private[spark] object JsonProtocol { case fetchFailed: FetchFailed => val blockManagerAddress = blockManagerIdToJson(fetchFailed.bmAddress) ("Block Manager Address" -> blockManagerAddress) ~ - ("Shuffle ID" -> fetchFailed.shuffleId) ~ - ("Map ID" -> fetchFailed.mapId) ~ - ("Reduce ID" -> fetchFailed.reduceId) + ("Shuffle ID" -> fetchFailed.shuffleId) ~ + ("Map ID" -> fetchFailed.mapId) ~ + ("Reduce ID" -> fetchFailed.reduceId) case exceptionFailure: ExceptionFailure => val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) ("Class Name" -> exceptionFailure.className) ~ - ("Description" -> exceptionFailure.description) ~ - ("Stack Trace" -> stackTrace) ~ - ("Metrics" -> metrics) + ("Description" -> exceptionFailure.description) ~ + ("Stack Trace" -> stackTrace) ~ + ("Metrics" -> metrics) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -235,9 +234,9 @@ private[spark] object JsonProtocol { def blockManagerIdToJson(blockManagerId: BlockManagerId): JValue = { ("Executor ID" -> blockManagerId.executorId) ~ - ("Host" -> blockManagerId.host) ~ - ("Port" -> blockManagerId.port) ~ - ("Netty Port" -> blockManagerId.nettyPort) + ("Host" -> blockManagerId.host) ~ + ("Port" -> blockManagerId.port) ~ + ("Netty Port" -> blockManagerId.nettyPort) } def jobResultToJson(jobResult: JobResult): JValue = { @@ -247,7 +246,7 @@ private[spark] object JsonProtocol { case jobFailed: JobFailed => val exception = exceptionToJson(jobFailed.exception) ("Exception" -> exception) ~ - ("Failed Stage ID" -> jobFailed.failedStageId) + ("Failed Stage ID" -> jobFailed.failedStageId) } ("Result" -> result) ~ json } @@ -257,29 +256,29 @@ private[spark] object JsonProtocol { val blocks = JArray( storageStatus.blocks.toList.map { case (id, status) => ("Block ID" -> blockIdToJson(id)) ~ - ("Status" -> blockStatusToJson(status)) + ("Status" -> blockStatusToJson(status)) }) ("Block Manager ID" -> blockManagerId) ~ - ("Maximum Memory" -> storageStatus.maxMem) ~ - ("Blocks" -> blocks) + ("Maximum Memory" -> storageStatus.maxMem) ~ + ("Blocks" -> blocks) } def rddInfoToJson(rddInfo: RDDInfo): JValue = { val storageLevel = storageLevelToJson(rddInfo.storageLevel) ("RDD ID" -> rddInfo.id) ~ - ("Name" -> rddInfo.name) ~ - ("Storage Level" -> storageLevel) ~ - ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ - ("Number of Partitions" -> rddInfo.numPartitions) ~ - ("Memory Size" -> rddInfo.memSize) ~ - ("Disk Size" -> rddInfo.diskSize) + ("Name" -> rddInfo.name) ~ + ("Storage Level" -> storageLevel) ~ + ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ + ("Number of Partitions" -> rddInfo.numPartitions) ~ + ("Memory Size" -> rddInfo.memSize) ~ + ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ - ("Use Memory" -> storageLevel.useMemory) ~ - ("Deserialized" -> storageLevel.deserialized) ~ - ("Replication" -> storageLevel.replication) + ("Use Memory" -> storageLevel.useMemory) ~ + ("Deserialized" -> storageLevel.deserialized) ~ + ("Replication" -> storageLevel.replication) } def blockIdToJson(blockId: BlockId): JValue = { @@ -287,21 +286,21 @@ private[spark] object JsonProtocol { val json: JObject = blockId match { case rddBlockId: RDDBlockId => ("RDD ID" -> rddBlockId.rddId) ~ - ("Split Index" -> rddBlockId.splitIndex) + ("Split Index" -> rddBlockId.splitIndex) case shuffleBlockId: ShuffleBlockId => ("Shuffle ID" -> shuffleBlockId.shuffleId) ~ - ("Map ID" -> shuffleBlockId.mapId) ~ - ("Reduce ID" -> shuffleBlockId.reduceId) + ("Map ID" -> shuffleBlockId.mapId) ~ + ("Reduce ID" -> shuffleBlockId.reduceId) case broadcastBlockId: BroadcastBlockId => "Broadcast ID" -> broadcastBlockId.broadcastId case broadcastHelperBlockId: BroadcastHelperBlockId => ("Broadcast Block ID" -> blockIdToJson(broadcastHelperBlockId.broadcastId)) ~ - ("Helper Type" -> broadcastHelperBlockId.hType) + ("Helper Type" -> broadcastHelperBlockId.hType) case taskResultBlockId: TaskResultBlockId => "Task ID" -> taskResultBlockId.taskId case streamBlockId: StreamBlockId => ("Stream ID" -> streamBlockId.streamId) ~ - ("Unique ID" -> streamBlockId.uniqueId) + ("Unique ID" -> streamBlockId.uniqueId) case tempBlockId: TempBlockId => val uuid = UUIDToJson(tempBlockId.id) "Temp ID" -> uuid @@ -314,8 +313,8 @@ private[spark] object JsonProtocol { def blockStatusToJson(blockStatus: BlockStatus): JValue = { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ - ("Memory Size" -> blockStatus.memSize) ~ - ("Disk Size" -> blockStatus.diskSize) + ("Memory Size" -> blockStatus.memSize) ~ + ("Disk Size" -> blockStatus.diskSize) } /** @@ -335,21 +334,21 @@ private[spark] object JsonProtocol { def UUIDToJson(id: UUID): JValue = { ("Least Significant Bits" -> id.getLeastSignificantBits) ~ - ("Most Significant Bits" -> id.getMostSignificantBits) + ("Most Significant Bits" -> id.getMostSignificantBits) } def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { JArray(stackTrace.map { case line => ("Declaring Class" -> line.getClassName) ~ - ("Method Name" -> line.getMethodName) ~ - ("File Name" -> line.getFileName) ~ - ("Line Number" -> line.getLineNumber) + ("Method Name" -> line.getMethodName) ~ + ("File Name" -> line.getFileName) ~ + ("Line Number" -> line.getLineNumber) }.toList) } def exceptionToJson(exception: Exception): JValue = { ("Message" -> exception.toString) ~ - ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) + ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) } /** @@ -699,19 +698,19 @@ private[spark] object JsonProtocol { def UUIDFromJson(json: JValue): UUID = { implicit val format = DefaultFormats - new UUID( - (json \ "Least Significant Bits").extract[Long], - (json \ "Most Significant Bits").extract[Long]) + val leastSignificantBits = (json \ "Least Significant Bits").extract[Long] + val mostSignificantBits = (json \ "Most Significant Bits").extract[Long] + new UUID(leastSignificantBits, mostSignificantBits) } def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { implicit val format = DefaultFormats json.extract[List[JValue]].map { line => - new StackTraceElement( - (line \ "Declaring Class").extract[String], - (line \ "Method Name").extract[String], - (line \ "File Name").extract[String], - (line \ "Line Number").extract[Int]) + val declaringClass = (line \ "Declaring Class").extract[String] + val methodName = (line \ "Method Name").extract[String] + val fileName = (line \ "File Name").extract[String] + val lineNumber = (line \ "Line Number").extract[Int] + new StackTraceElement(declaringClass, methodName, fileName, lineNumber) }.toArray } From bbe3501c63029ffa9c1fd9053e7ab868d0f28b10 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 26 Feb 2014 15:27:43 -0800 Subject: [PATCH 29/68] Embed storage status and RDD info in Task events This commit achieves three main things. First and foremost, it embeds the information from the SparkListenerFetchStorageStatus and SparkListenerGetRDDInfo events into events that are more descriptive of the SparkListenerInterface. In particular, every Task now maintains a list of blocks whose storage status have been updated as a result of the task. Previously, this information is retrieved from fetching storage status from the driver, an action arbitrarily associated with a stage. This change involves keeping track of what blocks are dropped during each call to an RDD persist. A big TODO is to also capture the behavior of an RDD unpersist in a SparkListenerEvent. Second, the SparkListenerEvent interface now handles the dynamic nature of Executors. In particular, a new event, SparkListenerExecutorStateChange, is introduced, which triggers a storage status fetch from the driver. The purpose of this is mainly to decouple fetching storage status from the driver from the Stage. Note that storage status is not ready until the remote BlockManagers have been registered, so this involves attaching a registration listener to the BlockManagerMasterActor. Third, changes in environment properties is now supported. This accounts for the fact that the user can invoke sc.addFile and sc.addJar in his/her own application, which should be reflected appropriately on the EnvironmentUI. In the previous implementation, coupling this information with application start prevents this from happening. Other relatively minor changes include: 1) Refactoring BlockStatus and BlockManagerInfo to not be a part of the BlockManagerMasterActor object, 2) Formatting changes, especially those involving multi-line arguments, and 3) Making all UI widgets and listeners private[ui] instead of private[spark]. --- .../scala/org/apache/spark/CacheManager.scala | 16 +- .../scala/org/apache/spark/SparkContext.scala | 49 +++- .../scala/org/apache/spark/SparkEnv.scala | 48 ++-- .../apache/spark/deploy/worker/Worker.scala | 1 - .../apache/spark/executor/TaskMetrics.scala | 7 + .../apache/spark/scheduler/DAGScheduler.scala | 32 ++- .../spark/scheduler/SchedulerBackend.scala | 2 - .../spark/scheduler/SparkListener.scala | 22 +- .../spark/scheduler/SparkListenerBus.scala | 8 +- .../apache/spark/scheduler/StageInfo.scala | 9 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 - .../apache/spark/storage/BlockManager.scala | 227 +++++++++++------- .../spark/storage/BlockManagerMaster.scala | 4 +- .../storage/BlockManagerMasterActor.scala | 157 ++++++------ .../BlockManagerRegistrationListener.scala | 51 ++++ .../apache/spark/storage/MemoryStore.scala | 82 ++++--- .../org/apache/spark/storage/PutResult.scala | 12 +- .../apache/spark/storage/StorageUtils.scala | 114 ++++++--- .../org/apache/spark/ui/UISparkListener.scala | 137 ++++++----- .../apache/spark/ui/env/EnvironmentUI.scala | 11 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 14 +- .../spark/ui/jobs/ExecutorSummary.scala | 2 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 32 ++- .../apache/spark/ui/jobs/JobProgressUI.scala | 4 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../spark/ui/storage/BlockManagerUI.scala | 37 +-- .../apache/spark/ui/storage/IndexPage.scala | 4 +- .../org/apache/spark/ui/storage/RDDPage.scala | 19 +- .../org/apache/spark/util/JsonProtocol.scala | 171 ++++++------- .../spark/scheduler/SparkListenerSuite.scala | 10 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- 36 files changed, 754 insertions(+), 544 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index bca74825efd28..85abfd8e65045 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -18,13 +18,15 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} + import org.apache.spark.storage.{BlockManager, StorageLevel, RDDBlockId} import org.apache.spark.rdd.RDD -/** Spark class responsible for passing RDDs split contents to the BlockManager and making - sure a node doesn't load two copies of an RDD at once. - */ +/** + * Spark class responsible for passing RDDs split contents to the BlockManager and making + * sure a node doesn't load two copies of an RDD at once. + */ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD splits that are being computed/loaded. */ @@ -69,11 +71,17 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If we got here, we have to load the split logInfo("Partition %s not found, computing it".format(key)) val computedValues = rdd.computeOrReadCheckpoint(split, context) + // Persist the result, so long as the task is not running locally if (context.runningLocally) { return computedValues } val elements = new ArrayBuffer[Any] elements ++= computedValues - blockManager.put(key, elements, storageLevel, tellMaster = true) + val updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) + + // Update task metrics to include any updated blocks + val metrics = context.taskMetrics + metrics.updatedBlocks = Some(updatedBlocks ++ metrics.updatedBlocks.getOrElse(Seq())) + elements.iterator.asInstanceOf[Iterator[T]] } finally { loading.synchronized { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d024bcddb21f4..a43fd6b2f0c63 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -161,7 +161,7 @@ class SparkContext( // Add each JAR given through the constructor if (jars != null) { - jars.foreach(addJar) + jars.foreach { jar => addJar(jar, updateEnvironment = false) } } private[spark] val executorMemory = conf.getOption("spark.executor.memory") @@ -204,14 +204,14 @@ class SparkContext( taskScheduler.start() @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) - dagScheduler.start() + // Start the UI before the DAG scheduler, because the UI listens for Spark events ui.start() - // Trigger application start - val environmentDetails = SparkEnv.environmentDetails(this) - val applicationStart = new SparkListenerApplicationStart(environmentDetails) - dagScheduler.post(applicationStart) + dagScheduler.start() + dagScheduler.post(new SparkListenerApplicationStart(appName)) + + updateEnvironmentProperties() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -631,7 +631,7 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, * use `SparkFiles.get(path)` to find its download location. */ - def addFile(path: String) { + def addFile(path: String, updateEnvironment: Boolean = true) { val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addFile(new File(uri.getPath)) @@ -644,6 +644,9 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) + if (updateEnvironment) { + updateEnvironmentProperties() + } } def addSparkListener(listener: SparkListener) { @@ -711,8 +714,11 @@ class SparkContext( * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ - def clearFiles() { + def clearFiles(updateEnvironment: Boolean = true) { addedFiles.clear() + if (updateEnvironment) { + updateEnvironmentProperties() + } } /** @@ -730,7 +736,7 @@ class SparkContext( * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported * filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node. */ - def addJar(path: String) { + def addJar(path: String, updateEnvironment: Boolean = true) { if (path == null) { logWarning("null specified as parameter to addJar") } else { @@ -774,14 +780,20 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } + if (updateEnvironment) { + updateEnvironmentProperties() + } } /** * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ - def clearJars() { + def clearJars(updateEnvironment: Boolean = true) { addedJars.clear() + if (updateEnvironment) { + updateEnvironmentProperties() + } } /** Shut down the SparkContext. */ @@ -798,8 +810,8 @@ class SparkContext( // TODO: Cache.stop()? env.stop() // Clean up locally linked files - clearFiles() - clearJars() + clearFiles(updateEnvironment = false) + clearJars(updateEnvironment = false) SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() @@ -1022,6 +1034,19 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Update environment properties and post the corresponding event to the DAG scheduler */ + private def updateEnvironmentProperties() { + val schedulingMode = getSchedulingMode.toString + val addedJarPaths = addedJars.keys.toSeq + val addedFilePaths = addedFiles.keys.toSeq + val environmentDetails = + SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) + val environmentUpdate = new SparkListenerEnvironmentUpdate(environmentDetails) + + // In case the DAG scheduler is not ready yet, first check whether its reference is valid + Option(dagScheduler).foreach(_.post(environmentUpdate)) + } + /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ff3f25388a631..86db2d0eb7f67 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -24,15 +24,15 @@ import scala.util.Properties import akka.actor._ +import com.google.common.collect.MapMaker + +import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster} import org.apache.spark.network.ConnectionManager import org.apache.spark.serializer.{Serializer, SerializerManager} +import org.apache.spark.storage._ import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.api.python.PythonWorkerFactory - -import com.google.common.collect.MapMaker /** * Holds all the runtime environment objects for a running Spark instance (either master or worker), @@ -167,9 +167,18 @@ object SparkEnv extends Logging { } } - val blockManagerMaster = new BlockManagerMaster(registerOrLookup( - "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf)), conf) + // Listen for block manager registration + val blockManagerListener = new BlockManagerRegistrationListener + lazy val blockManagerMasterActor = { + val actor = new BlockManagerMasterActor(isLocal, conf) + actor.registerListener(blockManagerListener) + actor + } + + val blockManagerMaster = + new BlockManagerMaster(registerOrLookup("BlockManagerMaster", blockManagerMasterActor), conf) + blockManagerMaster.registrationListener = Some(blockManagerListener) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) @@ -243,7 +252,12 @@ object SparkEnv extends Logging { * attributes as a sequence of KV pairs. */ private[spark] - def environmentDetails(sc: SparkContext): Map[String, Seq[(String, String)]] = { + def environmentDetails( + conf: SparkConf, + schedulingMode: String, + addedJars: Seq[String], + addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), ("Java Home", Properties.javaHome), @@ -251,15 +265,12 @@ object SparkEnv extends Logging { ("Scala Home", Properties.scalaHome) ).sorted - // Spark properties, including scheduling mode and app name whether or not they are configured + // Spark properties, including scheduling mode whether or not it is configured var additionalFields = Seq[(String, String)]() - sc.conf.getOption("spark.scheduler.mode").getOrElse { - additionalFields ++= Seq(("spark.scheduler.mode", sc.getSchedulingMode.toString)) - } - sc.conf.getOption("spark.app.name").getOrElse { - additionalFields ++= Seq(("spark.app.name", sc.appName)) + conf.getOption("spark.scheduler.mode").getOrElse { + additionalFields ++= Seq(("spark.scheduler.mode", schedulingMode)) } - val sparkProperties = sc.conf.getAll.sorted ++ additionalFields + val sparkProperties = conf.getAll.sorted ++ additionalFields val systemProperties = System.getProperties.iterator.toSeq val classPathProperty = systemProperties.find { case (k, v) => @@ -273,12 +284,11 @@ object SparkEnv extends Logging { // Class paths including all added jars and files val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) + .split(conf.get("path.separator", ":")) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val addedFiles = sc.addedFiles.iterator.toSeq.map{ case (path, _) => (path, "Added By User") } - val classPaths = (addedJars ++ addedFiles ++ classPathEntries).sorted + val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) + val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted Map[String, Seq[(String, String)]]( "JVM Information" -> jvmInformation, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f4ee0e2343849..7b0b7861b76e1 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -31,7 +31,6 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index bf421da04d6fe..8b6ed9e38e99f 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import org.apache.spark.storage.{BlockId, BlockStatus} + class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -68,6 +70,11 @@ class TaskMetrics extends Serializable { * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + /** + * If blocks have been updated as a result of this task, collect the statuses of this blocks here + */ + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } object TaskMetrics { 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 74b7e501954ba..b9c1c236f45eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -194,6 +194,9 @@ class DAGScheduler( } } })) + + // Start listening for block manager registration + blockManagerMaster.registrationListener.foreach(_.setListenerBus(listenerBus)) } def addSparkListener(listener: SparkListener) { @@ -545,14 +548,13 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) + post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -593,15 +595,15 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) } } - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + post(SparkListenerTaskStart(task.stageId, taskInfo)) case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) + post(SparkListenerTaskGettingResult(taskInfo)) case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) + post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -619,7 +621,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) + post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) } return true } @@ -697,7 +699,7 @@ class DAGScheduler( stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through stageToInfos -= s // completion events or stage abort jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) + post(SparkListenerJobEnd(job.jobId, jobResult)) } } @@ -771,7 +773,7 @@ class DAGScheduler( // must be run listener before possible NotSerializableException // should be "StageSubmitted" first and then "JobEnded" - listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) + post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) if (tasks.size > 0) { // Preemptively serialize a task to make sure it can be serialized. We are catching this @@ -820,7 +822,7 @@ class DAGScheduler( } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + post(SparkListenerStageCompleted(stageToInfos(stage))) running -= stage } event.reason match { @@ -845,7 +847,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) jobIdToStageIdsRemove(job.jobId) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) + post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -982,6 +984,8 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } + val storageStatusList = blockManagerMaster.getStorageStatus + post(new SparkListenerExecutorsStateChange(storageStatusList)) } private def handleExecutorGained(execId: String, host: String) { @@ -990,6 +994,8 @@ class DAGScheduler( logInfo("Host gained which was in lost list earlier: " + host) failedEpoch -= execId } + // Do not trigger SparkListenerExecutorsStateChange, because it is already triggered in + // blockManagerMaster.registrationListener when a new BlockManager registers with the master } private def handleJobCancellation(jobId: Int) { @@ -1004,7 +1010,7 @@ class DAGScheduler( jobIdToStageIds -= jobId activeJobs -= job idToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1027,7 +1033,7 @@ class DAGScheduler( idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 02bdbba825781..eefc8c232b564 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.SparkContext - /** * A backend interface for scheduling systems that allows plugging in different ones under * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as 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 a492cfc07cd78..6ae05e75b6387 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.Map import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.storage.{RDDInfo, StorageStatus} +import org.apache.spark.storage.StorageStatus sealed trait SparkListenerEvent @@ -45,19 +45,16 @@ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Pro case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -case class SparkListenerApplicationStart(environmentDetails: Map[String, Seq[(String, String)]]) - extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String) extends SparkListenerEvent -/** An event used in the ExecutorsUI and BlockManagerUI to fetch storage status from SparkEnv */ -private[spark] case class SparkListenerStorageStatusFetch(storageStatusList: Seq[StorageStatus]) +case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -/** An event used in the BlockManagerUI to query information of persisted RDDs */ -private[spark] case class SparkListenerGetRDDInfo(rddInfoList: Seq[RDDInfo]) +case class SparkListenerExecutorsStateChange(storageStatusList: Seq[StorageStatus]) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvent +private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** @@ -106,14 +103,15 @@ trait SparkListener { def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } /** - * Called when Spark fetches storage statuses from the driver + * Called when environment properties have been updated */ - def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { } + def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } /** - * Called when Spark queries statuses of persisted RDD's + * Called when a new executor has joined, or an existing executor is lost */ - def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { } + def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { } + } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 2eb57996b4feb..3a7d0ff9408a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -71,10 +71,10 @@ private[spark] class SparkListenerBus extends Logging { listeners.foreach(_.onTaskEnd(taskEnd)) case applicationStart: SparkListenerApplicationStart => listeners.foreach(_.onApplicationStart(applicationStart)) - case storageStatusFetch: SparkListenerStorageStatusFetch => - listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) - case getRDDInfo: SparkListenerGetRDDInfo => - listeners.foreach(_.onGetRDDInfo(getRDDInfo)) + case environmentUpdate: SparkListenerEnvironmentUpdate => + listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case executorsStateChange: SparkListenerExecutorsStateChange => + listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) case SparkListenerShutdown => return true case _ => diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 6cfe34e836f74..577932474c1fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.RDDInfo /** * Stores information about a stage to pass from the scheduler to SparkListeners. Also @@ -29,9 +30,8 @@ private[spark] class StageInfo( val stageId: Int, val name: String, - val rddName: String, - val numPartitions: Int, val numTasks: Int, + val rddInfo: RDDInfo, val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]()) { @@ -44,6 +44,9 @@ class StageInfo( private[spark] object StageInfo { def fromStage(stage: Stage): StageInfo = { - new StageInfo(stage.id, stage.name, stage.rdd.name, stage.numPartitions, stage.numTasks) + val rdd = stage.rdd + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 04f35cca08262..016145dfde57f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler.cluster -import scala.collection.mutable.HashMap - import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 47ea144c0c799..8d32ceda32ae1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -21,22 +21,22 @@ import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{HashMap, ArrayBuffer} +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} -import scala.concurrent.{Await, Future} -import scala.concurrent.duration._ import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} +import sun.nio.ch.DirectBuffer + import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -import sun.nio.ch.DirectBuffer - private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, @@ -87,7 +87,7 @@ private[spark] class BlockManager( val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - // Pending reregistration action being executed asynchronously or null if none + // Pending re-registration action being executed asynchronously or null if none // is pending. Accesses should synchronize on asyncReregisterLock. var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object @@ -117,8 +117,12 @@ private[spark] class BlockManager( /** * Construct a BlockManager with a memory limit set based on system properties. */ - def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf) = { + def this( + execId: String, + actorSystem: ActorSystem, + master: BlockManagerMaster, + serializer: Serializer, + conf: SparkConf) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf) } @@ -142,14 +146,15 @@ private[spark] class BlockManager( * an executor crash. * * This function deliberately fails silently if the master returns false (indicating that - * the slave needs to reregister). The error condition will be detected again by the next - * heart beat attempt or new block registration and another try to reregister all blocks + * the slave needs to re-register). The error condition will be detected again by the next + * heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ private def reportAllBlocks() { logInfo("Reporting " + blockInfo.size + " blocks to the master.") for ((blockId, info) <- blockInfo) { - if (!tryToReportBlockStatus(blockId, info)) { + val status = getUpdatedBlockStatus(blockId, info) + if (!tryToReportBlockStatus(blockId, info, status)) { logError("Failed to report " + blockId + " to master; giving up.") return } @@ -157,20 +162,20 @@ private[spark] class BlockManager( } /** - * Reregister with the master and report all blocks to it. This will be called by the heart beat + * Re-register with the master and report all blocks to it. This will be called by the heart beat * thread if our heartbeat to the block manager indicates that we were not registered. * * Note that this method must be called without any BlockInfo locks held. */ def reregister() { - // TODO: We might need to rate limit reregistering. - logInfo("BlockManager reregistering with master") + // TODO: We might need to rate limit re-registering. + logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) reportAllBlocks() } /** - * Reregister with the master sometime soon. + * Re-register with the master sometime soon. */ def asyncReregister() { asyncReregisterLock.synchronized { @@ -186,7 +191,7 @@ private[spark] class BlockManager( } /** - * For testing. Wait for any pending asynchronous reregistration; otherwise, do nothing. + * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. */ def waitForAsyncReregister() { val task = asyncReregisterTask @@ -205,15 +210,19 @@ private[spark] class BlockManager( * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. * - * droppedMemorySize exists to account for when block is dropped from memory to disk (so it - * is still valid). This ensures that update in master will compensate for the increase in + * droppedMemorySize exists to account for when the block is dropped from memory to disk (so + * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) { - val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) + def reportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, + droppedMemorySize: Long = 0L) { + val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) if (needReregister) { - logInfo("Got told to reregister updating block " + blockId) - // Reregistering will report our new block for free. + logInfo("Got told to re-register updating block " + blockId) + // Re-registering will report our new block for free. asyncReregister() } logDebug("Told master about block " + blockId) @@ -224,27 +233,41 @@ private[spark] class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, + private def tryToReportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + if (info.tellMaster) { + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) + } else { + true + } + } + + /** + * Return the updated storage status of the block with the given ID. More specifically, if + * the block is dropped from memory and possibly added to disk, return the new storage level + * and the updated in-memory and on-disk sizes. + */ + private def getUpdatedBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { + val (newLevel, inMemSize, onDiskSize) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, false) + (StorageLevel.NONE, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) - val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize + val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, info.tellMaster) + (storageLevel, memSize, diskSize) } } - - if (tellMaster) { - master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) - } else { - true - } + BlockStatus(newLevel, inMemSize, onDiskSize) } /** @@ -392,10 +415,10 @@ private[spark] class BlockManager( /** * Get block from remote block managers as serialized bytes. */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { + def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug("Getting remote block " + blockId + " as bytes") doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] - } + } private def doGetRemote(blockId: BlockId, asValues: Boolean): Option[Any] = { require(blockId != null, "BlockId is null") @@ -441,9 +464,8 @@ private[spark] class BlockManager( * so that we can control the maxMegabytesInFlight for the fetch. */ def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer) - : BlockFetcherIterator = { - + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer): BlockFetcherIterator = { val iter = if (conf.getBoolean("spark.shuffle.use.netty", false)) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) @@ -455,8 +477,11 @@ private[spark] class BlockManager( iter } - def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) - : Long = { + def put( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { val elements = new ArrayBuffer[Any] elements ++= values put(blockId, elements, level, tellMaster) @@ -476,32 +501,47 @@ private[spark] class BlockManager( } /** - * Put a new block of values to the block manager. Returns its (estimated) size in bytes. + * Put a new block of values to the block manager. Return a list of blocks updated as a + * result of this put. */ - def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - tellMaster: Boolean = true) : Long = { + def put( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") doPut(blockId, Left(values), level, tellMaster) } /** - * Put a new block of serialized bytes to the block manager. + * Put a new block of serialized bytes to the block manager. Return a list of blocks updated + * as a result of this put. */ - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true) { + def putBytes( + blockId: BlockId, + bytes: ByteBuffer, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") doPut(blockId, Right(bytes), level, tellMaster) } - private def doPut(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer], - level: StorageLevel, tellMaster: Boolean = true): Long = { + private def doPut( + blockId: BlockId, + data: Either[ArrayBuffer[Any], ByteBuffer], + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + // Return value + val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. - val myInfo = { + val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) @@ -509,7 +549,7 @@ private[spark] class BlockManager( if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return oldBlockOpt.get.size + return updatedBlocks } // TODO: So the block info exists - but previous attempt to load it (?) failed. @@ -531,7 +571,7 @@ private[spark] class BlockManager( // Ditto for the bytes after the put var bytesAfterPut: ByteBuffer = null - // Size of the block in bytes (to return to caller) + // Size of the block in bytes var size = 0L // If we're storing bytes, then initiate the replication before storing them locally. @@ -545,7 +585,7 @@ private[spark] class BlockManager( null } - myInfo.synchronized { + putBlockInfo.synchronized { logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") @@ -562,6 +602,8 @@ private[spark] class BlockManager( case Right(newBytes) => bytesAfterPut = newBytes case Left(newIterator) => valuesAfterPut = newIterator } + // Keep track of which blocks are dropped from memory + res.droppedBlocks.foreach { block => updatedBlocks += block } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -585,20 +627,23 @@ private[spark] class BlockManager( // Now that the block is in either the memory or disk store, let other threads read it, // and tell the master about it. marked = true - myInfo.markReady(size) + putBlockInfo.markReady(size) + val putBlockStatus = getUpdatedBlockStatus(blockId, putBlockInfo) if (tellMaster) { - reportBlockStatus(blockId, myInfo) - } - } finally { - // If we failed at putting the block to memory/disk, notify other possible readers - // that it has failed, and then remove it from the block info map. - if (! marked) { - // Note that the remove must happen before markFailure otherwise another thread - // could've inserted a new BlockInfo before we remove it. - blockInfo.remove(blockId) - myInfo.markFailure() - logWarning("Putting block " + blockId + " failed") + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } + updatedBlocks += ((blockId, putBlockStatus)) + } catch { + case e: Exception => + // If we failed in putting the block to memory/disk, notify other possible readers + // that it has failed, and then remove it from the block info map. + if (!marked) { + // Note that the remove must happen before markFailure otherwise another thread + // could've inserted a new BlockInfo before we remove it. + blockInfo.remove(blockId) + putBlockInfo.markFailure() + logWarning("Putting block " + blockId + " failed") + } } } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) @@ -635,7 +680,7 @@ private[spark] class BlockManager( Utils.getUsedTimeMs(startTimeMs)) } - size + updatedBlocks } /** @@ -679,21 +724,31 @@ private[spark] class BlockManager( /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. + * + * Return the block status if the given block has been updated, else None. */ - def dropFromMemory(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]) { + def dropFromMemory( + blockId: BlockId, + data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + logInfo("Dropping block " + blockId + " from memory") val info = blockInfo.get(blockId).orNull + + // If the block has not already been dropped if (info != null) { info.synchronized { // required ? As of now, this will be invoked only for blocks which are ready // But in case this changes in future, adding for consistency sake. - if (! info.waitForReady() ) { + if (!info.waitForReady()) { // If we get here, the block write failed. logWarning("Block " + blockId + " was marked as failure. Nothing to drop") - return + return None } + var blockIsUpdated = false val level = info.level + + // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { logInfo("Writing block " + blockId + " to disk") data match { @@ -702,24 +757,33 @@ private[spark] class BlockManager( case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } + blockIsUpdated = true } + + // Actually drop from memory store val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L - val blockWasRemoved = memoryStore.remove(blockId) - if (!blockWasRemoved) { + val blockIsRemoved = memoryStore.remove(blockId) + if (blockIsRemoved) { + blockIsUpdated = true + } else { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") } + + val status = getUpdatedBlockStatus(blockId, info) if (info.tellMaster) { - reportBlockStatus(blockId, info, droppedMemorySize) + reportBlockStatus(blockId, info, status, droppedMemorySize) } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. blockInfo.remove(blockId) } + if (blockIsUpdated) { + return Some(status) + } } - } else { - // The block has already been dropped } + None } /** @@ -751,7 +815,8 @@ private[spark] class BlockManager( } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, info) + val status = getUpdatedBlockStatus(blockId, info) + reportBlockStatus(blockId, info, status) } } else { // The block has already been removed; do nothing. @@ -786,7 +851,8 @@ private[spark] class BlockManager( iterator.remove() logInfo("Dropped block " + id) } - reportBlockStatus(id, info) + val status = getUpdatedBlockStatus(id, info) + reportBlockStatus(id, info, status) } } } @@ -896,9 +962,8 @@ private[spark] object BlockManager extends Logging { def blockIdsToBlockManagers( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[BlockManagerId]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + // blockManagerMaster != null is used in tests assert (env != null || blockManagerMaster != null) val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) { @@ -917,18 +982,14 @@ private[spark] object BlockManager extends Logging { def blockIdsToExecutorIds( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) } def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index c54e4f2664753..f1c363caed581 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,7 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { +class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) @@ -37,6 +37,8 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo val timeout = AkkaUtils.askTimeout(conf) + var registrationListener: Option[BlockManagerRegistrationListener] = None + /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { tell(RemoveExecutor(execId)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 893418fb8cad9..6a84a1217212f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.util.{HashMap => JHashMap} import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.concurrent.Future import scala.concurrent.duration._ @@ -39,8 +40,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { // Mapping from block manager id to the block manager's information. - private val blockManagerInfo = - new mutable.HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] // Mapping from executor ID to block manager ID. private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] @@ -50,6 +50,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) + private val listeners = new ArrayBuffer[BlockManagerRegistrationListener] + val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong @@ -67,6 +69,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act super.preStart() } + def registerListener(listener: BlockManagerRegistrationListener) = listeners += listener + def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) @@ -217,8 +221,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case(blockManagerId, info) => - import collection.JavaConverters._ - StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) + val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) + new StorageStatus(blockManagerId, info.maxMem, blockMap) }.toArray } @@ -233,9 +237,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act case None => blockManagerIdByExecutor(id.executorId) = id } - blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveActor) + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), + maxMemSize, slaveActor) } + listeners.foreach(_.onBlockManagerRegister(storageStatus)) } private def updateBlockInfo( @@ -307,97 +312,93 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } -private[spark] -object BlockManagerMasterActor { +private[spark] case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) - case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) +private[spark] class BlockManagerInfo( + val blockManagerId: BlockManagerId, + timeMs: Long, + val maxMem: Long, + val slaveActor: ActorRef) + extends Logging { - class BlockManagerInfo( - val blockManagerId: BlockManagerId, - timeMs: Long, - val maxMem: Long, - val slaveActor: ActorRef) - extends Logging { + private var _lastSeenMs: Long = timeMs + private var _remainingMem: Long = maxMem - private var _lastSeenMs: Long = timeMs - private var _remainingMem: Long = maxMem + // Mapping from block id to its status. + private val _blocks = new JHashMap[BlockId, BlockStatus] - // Mapping from block id to its status. - private val _blocks = new JHashMap[BlockId, BlockStatus] + logInfo("Registering block manager %s with %s RAM".format( + blockManagerId.hostPort, Utils.bytesToString(maxMem))) - logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.bytesToString(maxMem))) - - def updateLastSeenMs() { - _lastSeenMs = System.currentTimeMillis() - } + def updateLastSeenMs() { + _lastSeenMs = System.currentTimeMillis() + } - def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, + diskSize: Long) { - updateLastSeenMs() + updateLastSeenMs() - if (_blocks.containsKey(blockId)) { - // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + if (_blocks.containsKey(blockId)) { + // The block exists on the slave already. + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel - if (originalLevel.useMemory) { - _remainingMem += memSize - } + if (originalLevel.useMemory) { + _remainingMem += memSize } + } - if (storageLevel.isValid) { - // isValid means it is either stored in-memory or on-disk. - // But the memSize here indicates the data size in or dropped from memory, - // and the diskSize here indicates the data size in or dropped to disk. - // They can be both larger than 0, when a block is dropped from memory to disk. - // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. - if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) - _remainingMem -= memSize - logInfo("Added %s in memory on %s (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), - Utils.bytesToString(_remainingMem))) - } - if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) - logInfo("Added %s on disk on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) - } - } else if (_blocks.containsKey(blockId)) { - // If isValid is not true, drop the block. - val blockStatus: BlockStatus = _blocks.get(blockId) - _blocks.remove(blockId) - if (blockStatus.storageLevel.useMemory) { - _remainingMem += blockStatus.memSize - logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), - Utils.bytesToString(_remainingMem))) - } - if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) - } + if (storageLevel.isValid) { + // isValid means it is either stored in-memory or on-disk. + // But the memSize here indicates the data size in or dropped from memory, + // and the diskSize here indicates the data size in or dropped to disk. + // They can be both larger than 0, when a block is dropped from memory to disk. + // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. + if (storageLevel.useMemory) { + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _remainingMem -= memSize + logInfo("Added %s in memory on %s (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + Utils.bytesToString(_remainingMem))) + } + if (storageLevel.useDisk) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + logInfo("Added %s on disk on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) + } + } else if (_blocks.containsKey(blockId)) { + // If isValid is not true, drop the block. + val blockStatus: BlockStatus = _blocks.get(blockId) + _blocks.remove(blockId) + if (blockStatus.storageLevel.useMemory) { + _remainingMem += blockStatus.memSize + logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), + Utils.bytesToString(_remainingMem))) + } + if (blockStatus.storageLevel.useDisk) { + logInfo("Removed %s on %s on disk (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } } + } - def removeBlock(blockId: BlockId) { - if (_blocks.containsKey(blockId)) { - _remainingMem += _blocks.get(blockId).memSize - _blocks.remove(blockId) - } + def removeBlock(blockId: BlockId) { + if (_blocks.containsKey(blockId)) { + _remainingMem += _blocks.get(blockId).memSize + _blocks.remove(blockId) } + } - def remainingMem: Long = _remainingMem + def remainingMem: Long = _remainingMem - def lastSeenMs: Long = _lastSeenMs + def lastSeenMs: Long = _lastSeenMs - def blocks: JHashMap[BlockId, BlockStatus] = _blocks + def blocks: JHashMap[BlockId, BlockStatus] = _blocks - override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem + override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem - def clear() { - _blocks.clear() - } + def clear() { + _blocks.clear() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala new file mode 100644 index 0000000000000..1825198c7741f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala @@ -0,0 +1,51 @@ +/* + * 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.storage + +import org.apache.spark.scheduler._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable + +/** A listener for block manager state changes */ +private[spark] class BlockManagerRegistrationListener { + + private var _listenerBus: Option[SparkListenerBus] = None + + // Buffer any events received before the listener bus is ready + private val bufferedEvents = new ArrayBuffer[SparkListenerEvent] + with mutable.SynchronizedBuffer[SparkListenerEvent] + + /** + * Set the listener bus. If there are buffered events, post them all to the listener bus at once. + */ + def setListenerBus(listenerBus: SparkListenerBus) = { + _listenerBus = Some(listenerBus) + bufferedEvents.map(listenerBus.post) + } + + /** + * Called when a new BlockManager is registered with the master. If the listener bus is ready, + * post the event; otherwise, buffer it. + */ + def onBlockManagerRegister(storageStatus: Array[StorageStatus]) { + val executorsStateChange = new SparkListenerExecutorsStateChange(storageStatus) + _listenerBus.map(_.post(executorsStateChange)).getOrElse { + bufferedEvents += executorsStateChange + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 82089b923d190..c596c81446e81 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -19,7 +19,9 @@ package org.apache.spark.storage import java.util.LinkedHashMap import java.nio.ByteBuffer -import collection.mutable.ArrayBuffer + +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.{SizeEstimator, Utils} /** @@ -66,17 +68,15 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { - + returnValues: Boolean): PutResult = { if (level.deserialized) { val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) - tryToPut(blockId, values, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.iterator)) + val putAttempt = tryToPut(blockId, values, sizeEstimate, true) + PutResult(sizeEstimate, Left(values.iterator), putAttempt.droppedBlocks) } else { val bytes = blockManager.dataSerialize(blockId, values.iterator) - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) + val putAttempt = tryToPut(blockId, bytes, bytes.limit, false) + PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } @@ -141,19 +141,34 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) * size must also be passed by the caller. * - * Locks on the object putLock to ensure that all the put requests and its associated block + * Lock on the object putLock to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. + * + * Return whether put was successful, along with the blocks dropped in the process */ - private def tryToPut(blockId: BlockId, value: Any, size: Long, deserialized: Boolean): Boolean = { + private def tryToPut( + blockId: BlockId, + value: Any, + size: Long, + deserialized: Boolean): ResultWithDroppedBlocks = { + // TODO: Its possible to optimize the locking by locking entries only when selecting blocks - // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has been - // released, it must be ensured that those to-be-dropped blocks are not double counted for - // freeing up more space for another block that needs to be put. Only then the actually dropping - // of blocks (and writing to disk if necessary) can proceed in parallel. + // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has + // been released, it must be ensured that those to-be-dropped blocks are not double counted + // for freeing up more space for another block that needs to be put. Only then the actually + // dropping of blocks (and writing to disk if necessary) can proceed in parallel. + + var putSuccess = false + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + putLock.synchronized { - if (ensureFreeSpace(blockId, size)) { + val freeSpaceResult = ensureFreeSpace(blockId, size) + val enoughFreeSpace = freeSpaceResult.success + droppedBlocks ++= freeSpaceResult.droppedBlocks + + if (enoughFreeSpace) { val entry = new Entry(value, size, deserialized) entries.synchronized { entries.put(blockId, entry) @@ -166,7 +181,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) } - true + putSuccess = true } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. @@ -175,29 +190,33 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) - false + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } + ResultWithDroppedBlocks(putSuccess, droppedBlocks) } /** - * Tries to free up a given amount of space to store a particular block, but can fail and return - * false if either the block is bigger than our memory or it would require replacing another - * block from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that + * Try to free up a given amount of space to store a particular block, but can fail if + * either the block is bigger than our memory or it would require replacing another block + * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assumes that a lock is held by the caller to ensure only one thread is dropping blocks. + * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. * Otherwise, the freed space may fill up before the caller puts in their new value. + * + * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): Boolean = { - + private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( space, currentMemory, maxMemory)) + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + if (space > maxMemory) { logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } if (maxMemory - currentMemory < space) { @@ -216,7 +235,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (rddToAdd.isDefined && rddToAdd == getRddId(blockId)) { logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " + "block from the same RDD") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } selectedBlocks += blockId selectedMemory += pair.getValue.size @@ -236,15 +255,16 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } - return true + return ResultWithDroppedBlocks(success = true, droppedBlocks) } else { - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } } - true + ResultWithDroppedBlocks(success = true, droppedBlocks) } override def contains(blockId: BlockId): Boolean = { @@ -252,3 +272,5 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } +private case class ResultWithDroppedBlocks(success: Boolean, + droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala index 2eba2f06b5bfd..cd3f61e75d574 100644 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -20,7 +20,13 @@ package org.apache.spark.storage import java.nio.ByteBuffer /** - * Result of adding a block into a BlockStore. Contains its estimated size, and possibly the - * values put if the caller asked for them to be returned (e.g. for chaining replication) + * Result of adding a block into a BlockStore. This case class contains a few things: + * (1) The estimated size of the put, + * (2) The values put if the caller asked for them to be returned (e.g. for chaining + * replication), and + * (3) A list of blocks dropped as a result of this put. This is always empty for DiskStore. */ -private[spark] case class PutResult(size: Long, data: Either[Iterator[_], ByteBuffer]) +private[spark] case class PutResult( + size: Long, + data: Either[Iterator[_], ByteBuffer], + droppedBlocks: Seq[(BlockId, BlockStatus)] = Seq()) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 0126229686aa2..d89d5061e4849 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,14 +17,18 @@ package org.apache.spark.storage +import scala.collection.mutable +import scala.collection.Map + import org.apache.spark.SparkContext import org.apache.spark.util.Utils -import BlockManagerMasterActor.BlockStatus private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, - blocks: Map[BlockId, BlockStatus]) { +class StorageStatus( + val blockManagerId: BlockManagerId, + val maxMem: Long, + val blocks: mutable.Map[BlockId, BlockStatus]) { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) @@ -44,9 +48,14 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } } -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) +private[spark] +class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + override def toString = { ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, @@ -62,55 +71,80 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Returns RDD-level information, compiled from a list of StorageStatus objects */ - def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], - sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList( - storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) + /** Returns RDD-level information from a list of StorageStatus objects and SparkContext */ + def rddInfoFromStorageStatus( + storageStatusList: Seq[StorageStatus], + sc: SparkContext) : Array[RDDInfo] = { + val blockStatusMap = blockStatusMapFromStorageStatus(storageStatusList) + val rddInfoList = rddInfoFromSparkContext(blockStatusMap.keys.toSeq, sc) + val rddInfoMap = rddInfoList.map { info => (info.id, info) }.toMap + rddInfoFromBlockStatusMap(blockStatusMap, rddInfoMap) } - /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ - def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) - blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + /** + * Returns RDD-level information from a list of StorageStatus objects and an existing + * RDD ID to RDDInfo mapping + */ + def rddInfoFromStorageStatus( + storageStatusList: Seq[StorageStatus], + rddInfoMap: Map[Int, RDDInfo]): Array[RDDInfo] = { + val blockStatusMap = blockStatusMapFromStorageStatus(storageStatusList) + rddInfoFromBlockStatusMap(blockStatusMap, rddInfoMap) } - /* Given a list of BlockStatus objects, returns information for each RDD */ - def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus], - sc: SparkContext) : Array[RDDInfo] = { + private def rddInfoFromBlockStatusMap( + blockStatusMap: Map[Int, Array[BlockStatus]], + rddInfoMap: Map[Int, RDDInfo]): Array[RDDInfo] = { + val rddInfos = blockStatusMap.map { case (rddId, blocks) => + // Add up memory and disk sizes + val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } + val memSize = persistedBlocks.map(_.memSize).reduceOption(_+_).getOrElse(0L) + val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + rddInfoMap.get(rddId).map { rddInfo => + rddInfo.numCachedPartitions = persistedBlocks.length + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo + } + }.flatten.toArray - // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case (k, v) => k.rddId }.mapValues(_.values.toArray) + scala.util.Sorting.quickSort(rddInfos) + rddInfos + } - // For each RDD, generate an RDDInfo object - val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) => - // Add up memory and disk sizes - val memSize = rddBlocks.map(_.memSize).reduce(_ + _) - val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) + private def blockStatusMapFromStorageStatus(storageStatusList: Seq[StorageStatus]) + : Map[Int, Array[BlockStatus]] = { + val rddBlockMap = storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus] + rddBlockMap.groupBy { case (k, v) => k.rddId }.mapValues(_.values.toArray) + } - // Get the friendly name and storage level for the RDD, if available + private def rddInfoFromSparkContext(rddIds: Seq[Int], sc: SparkContext): Array[RDDInfo] = { + rddIds.flatMap { rddId => sc.persistentRdds.get(rddId).map { r => val rddName = Option(r.name).getOrElse(rddId.toString) + val rddNumPartitions = r.partitions.size val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, - memSize, diskSize) + val rddInfo = new RDDInfo(rddId, rddName, rddNumPartitions, rddStorageLevel) + rddInfo } - }.flatten.toArray - - scala.util.Sorting.quickSort(rddInfos) - - rddInfos + }.toArray } - /* Filters storage status by a given RDD id. */ - def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int) - : Array[StorageStatus] = { + /** Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ + def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { + val blockLocationPairs = + storageStatusList.flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + } + /** Filters storage status by a given RDD id. */ + def filterStorageStatusByRDD( + storageStatusList: Seq[StorageStatus], + rddId: Int) : Array[StorageStatus] = { storageStatusList.map { status => - val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus] - //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _) - StorageStatus(status.blockManagerId, status.maxMem, newBlocks) - } + val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq + val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) + new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) + }.toArray } } diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 91e461c6fc7f1..25d8c3d5d001a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -17,17 +17,17 @@ package org.apache.spark.ui +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import net.liftweb.json.JsonAST._ + import org.apache.spark.scheduler._ -import org.apache.spark.SparkContext -import org.apache.spark.storage.StorageStatus +import org.apache.spark.storage._ import org.apache.spark.util.FileLogger import org.apache.spark.util.JsonProtocol -import net.liftweb.json.JsonAST._ - -private[spark] trait UISparkListener extends SparkListener +private[ui] trait UISparkListener extends SparkListener /** * A SparkListener that serves as an entry point for all events posted to the UI. @@ -42,7 +42,7 @@ private[spark] trait UISparkListener extends SparkListener * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized * from the event logs to all attached listeners. */ -private[spark] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends SparkListener { +private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends SparkListener { // Log events only if the UI is live private val logger: Option[FileLogger] = if (live) Some(new FileLogger()) else None @@ -50,26 +50,28 @@ private[spark] class GatewayUISparkListener(parent: SparkUI, live: Boolean) exte // Children listeners for which this gateway is responsible private val listeners = ArrayBuffer[UISparkListener]() - def registerSparkListener(listener: UISparkListener) = { - listeners += listener - } + def registerSparkListener(listener: UISparkListener) = listeners += listener /** Log the event as JSON */ - private def logEvent(event: SparkListenerEvent) { + private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = JsonProtocol.sparkEventToJson(event) logger.foreach(_.logLine(compactRender(eventJson))) + if (flushLogger) { + logger.foreach(_.flush()) + } } + private def closeLogger() = logger.foreach(_.close()) + private def restartLogger() = logger.foreach(_.start()) + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { listeners.foreach(_.onStageSubmitted(stageSubmitted)) - logEvent(stageSubmitted) - logger.foreach(_.flush()) + logEvent(stageSubmitted, flushLogger = true) } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { listeners.foreach(_.onStageCompleted(stageCompleted)) - logEvent(stageCompleted) - logger.foreach(_.flush()) + logEvent(stageCompleted, flushLogger = true) } override def onTaskStart(taskStart: SparkListenerTaskStart) { @@ -87,72 +89,93 @@ private[spark] class GatewayUISparkListener(parent: SparkUI, live: Boolean) exte override def onJobStart(jobStart: SparkListenerJobStart) { listeners.foreach(_.onJobStart(jobStart)) - logger.foreach(_.start()) + restartLogger() logEvent(jobStart) } override def onJobEnd(jobEnd: SparkListenerJobEnd) { listeners.foreach(_.onJobEnd(jobEnd)) logEvent(jobEnd) - logger.foreach(_.close()) + closeLogger() } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - // Retrieve app name from the application start event // For live UI's, this should be equivalent to sc.appName - val sparkProperties = applicationStart.environmentDetails("Spark Properties").toMap - val appName = sparkProperties.get("spark.app.name") - appName.foreach(parent.setAppName) - + parent.setAppName(applicationStart.appName) listeners.foreach(_.onApplicationStart(applicationStart)) - logEvent(applicationStart) - logger.foreach(_.flush()) + logEvent(applicationStart, flushLogger = true) } - override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { - listeners.foreach(_.onStorageStatusFetch(storageStatusFetch)) - logEvent(storageStatusFetch) - logger.foreach(_.flush()) + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + logEvent(environmentUpdate) } - override def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { - listeners.foreach(_.onGetRDDInfo(getRDDInfo)) - logEvent(getRDDInfo) - logger.foreach(_.flush()) + override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { + listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) + logEvent(executorsStateChange, flushLogger = true) } } /** - * A SparkListener that fetches storage information from SparkEnv. - * - * The frequency at which this occurs is by default every time a stage event is triggered. - * This needs not be the case, however; a stage can be arbitrarily long, so any failure - * in the middle of a stage causes the storage status for that stage to be lost. + * A UISparkListener that maintains executor storage status */ -private[spark] class StorageStatusFetchSparkListener( - sc: SparkContext, - gateway: GatewayUISparkListener, - live: Boolean) - extends UISparkListener { - var storageStatusList: Seq[StorageStatus] = Seq() - - /** - * Fetch storage information from SparkEnv, which involves a query to the driver. This is - * expensive and should be invoked sparingly. - */ - def fetchStorageStatus() { - if (live) { - // Fetch only this is a live UI - val storageStatus = sc.getExecutorStorageStatus - val event = new SparkListenerStorageStatusFetch(storageStatus) - gateway.onStorageStatusFetch(event) +private[ui] class StorageStatusSparkListener extends UISparkListener { + var storageStatusList = Seq[StorageStatus]() + + /** Update storage status list to reflect updated block statuses */ + def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { + val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) + filteredStatus.foreach { storageStatus => + updatedBlocks.foreach { case (blockId, updatedStatus) => + storageStatus.blocks(blockId) = updatedStatus + } } } - override def onStorageStatusFetch(storageStatusFetch: SparkListenerStorageStatusFetch) { - storageStatusList = storageStatusFetch.storageStatusList + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val execId = taskEnd.taskInfo.executorId + val updatedBlocks = taskEnd.taskMetrics.updatedBlocks.getOrElse(Seq()) + if (updatedBlocks.length > 0) { + updateStorageStatus(execId, updatedBlocks) + } } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = fetchStorageStatus() - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = fetchStorageStatus() + override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { + storageStatusList = executorsStateChange.storageStatusList + } +} + +/** + * A UISparkListener that maintains RDD information + */ +private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { + private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + + /** Filter RDD info to include only those with cached partitions */ + def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + + /** Update each RDD's info to reflect any updates to the RDD's storage status */ + private def updateRDDInfo() { + val updatedRDDInfoList = StorageUtils.rddInfoFromStorageStatus(storageStatusList, _rddInfoMap) + updatedRDDInfoList.foreach { info => _rddInfoMap(info.id) = info } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + super.onTaskEnd(taskEnd) + if (taskEnd.taskMetrics.updatedBlocks.isDefined) { + updateRDDInfo() + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + val rddInfo = stageSubmitted.stageInfo.rddInfo + _rddInfoMap(rddInfo.id) = rddInfo + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + // Remove all partitions that are no longer cached + // TODO(aor): Handle unpersist + _rddInfoMap.retain { case (id, info) => info.numCachedPartitions > 0 } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 3cc85f6629b76..798c13f65b862 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -23,13 +23,12 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment import org.apache.spark.ui._ -private[spark] class EnvironmentUI(parent: SparkUI) { +private[ui] class EnvironmentUI(parent: SparkUI) { val live = parent.live val sc = parent.sc @@ -40,7 +39,7 @@ private[spark] class EnvironmentUI(parent: SparkUI) { def start() { val gateway = parent.gatewayListener - _listener = Some(new EnvironmentListener(sc)) + _listener = Some(new EnvironmentListener()) gateway.registerSparkListener(listener) } @@ -78,14 +77,14 @@ private[spark] class EnvironmentUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the EnvironmentUI */ -private[spark] class EnvironmentListener(sc: SparkContext) extends UISparkListener { +private[ui] class EnvironmentListener extends UISparkListener { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() - override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - val environmentDetails = applicationStart.environmentDetails + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + val environmentDetails = environmentUpdate.environmentDetails jvmInformation = environmentDetails("JVM Information") sparkProperties = environmentDetails("Spark Properties") systemProperties = environmentDetails("System Properties") diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 13e332e1e20a0..58f79e1435e75 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -24,14 +24,14 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.{SparkContext, ExceptionFailure} +import org.apache.spark.ExceptionFailure import org.apache.spark.scheduler._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui._ import org.apache.spark.util.Utils -private[spark] class ExecutorsUI(parent: SparkUI) { +private[ui] class ExecutorsUI(parent: SparkUI) { val live = parent.live val sc = parent.sc @@ -42,7 +42,7 @@ private[spark] class ExecutorsUI(parent: SparkUI) { def start() { val gateway = parent.gatewayListener - _listener = Some(new ExecutorsListener(sc, gateway, live)) + _listener = Some(new ExecutorsListener()) gateway.registerSparkListener(listener) } @@ -51,7 +51,6 @@ private[spark] class ExecutorsUI(parent: SparkUI) { ) def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) @@ -164,11 +163,7 @@ private[spark] class ExecutorsUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the ExecutorsUI */ -private[spark] class ExecutorsListener( - sc: SparkContext, - gateway: GatewayUISparkListener, - live: Boolean) - extends StorageStatusFetchSparkListener(sc, gateway, live) { +private[ui] class ExecutorsListener extends StorageStatusSparkListener { val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() @@ -203,6 +198,7 @@ private[spark] class ExecutorsListener( executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten } } + super.onTaskEnd(taskEnd) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 64e22a30b48f9..1dfe1d4f1fa11 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs /** class for reporting aggregated metrics for each executors in stageUI */ -private[spark] class ExecutorSummary { +private[ui] class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index a37bc7f9aeb9d..49581bc6beb1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -23,7 +23,7 @@ import scala.xml.Node import org.apache.spark.util.Utils /** Page showing executor summary */ -private[spark] class ExecutorTable(stageId: Int, parent: JobProgressUI) { +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { private def listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f753741cb26ac..e9657cff34151 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ -private[spark] class IndexPage(parent: JobProgressUI) { +private[ui] class IndexPage(parent: JobProgressUI) { private val live = parent.live private val sc = parent.sc private def appName = parent.appName diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 2f4a3fd52f6cf..495ea62d75739 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -21,9 +21,9 @@ import scala.collection.mutable.{ListBuffer, HashMap} import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler._ -import org.apache.spark.ui.{GatewayUISparkListener, StorageStatusFetchSparkListener} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler._ +import org.apache.spark.ui.StorageStatusSparkListener /** * Tracks task-level information to be displayed in the UI. @@ -32,11 +32,9 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[spark] class JobProgressListener( - sc: SparkContext, - gateway: GatewayUISparkListener, - live: Boolean) - extends StorageStatusFetchSparkListener(sc, gateway, live) { +private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) + extends StorageStatusSparkListener { + import JobProgressListener._ // How many stages to remember @@ -75,7 +73,7 @@ private[spark] class JobProgressListener( override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo val stageId = stage.stageId - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from disk + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) completedStages += stage @@ -172,7 +170,7 @@ private[spark] class JobProgressListener( } val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from disk + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage tasksActive.remove(taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = @@ -220,7 +218,7 @@ private[spark] class JobProgressListener( jobEnd.jobResult match { case JobFailed(_, stageId) => activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from disk + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage activeStages.remove(s.stageId) poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) failedStages += s @@ -230,9 +228,9 @@ private[spark] class JobProgressListener( } } - override def onApplicationStart(applicationStart: SparkListenerApplicationStart) = synchronized { + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) = { val schedulingModeName = - applicationStart.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") + environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") schedulingMode = schedulingModeName match { case Some(name) => Some(SchedulingMode.withName(name)) case None => None @@ -240,12 +238,12 @@ private[spark] class JobProgressListener( } } -private[spark] case class TaskUIData( - taskInfo: TaskInfo, - taskMetrics: Option[TaskMetrics] = None, - exception: Option[ExceptionFailure] = None) +private[ui] case class TaskUIData( + taskInfo: TaskInfo, + taskMetrics: Option[TaskMetrics] = None, + exception: Option[ExceptionFailure] = None) -private[spark] object JobProgressListener { +private object JobProgressListener { val DEFAULT_RETAINED_STAGES = 1000 val DEFAULT_POOL_NAME = "default" } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index a0e681c777198..2e693caa2c0f3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,7 +29,7 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(parent: SparkUI) { +private[ui] class JobProgressUI(parent: SparkUI) { val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val live = parent.live val sc = parent.sc @@ -45,7 +45,7 @@ private[spark] class JobProgressUI(parent: SparkUI) { def start() { val gateway = parent.gatewayListener - _listener = Some(new JobProgressListener(sc, gateway, live)) + _listener = Some(new JobProgressListener(sc, live)) gateway.registerSparkListener(listener) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 29bef73ed9b50..431b4515c11f7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressUI) { private val live = parent.live private val sc = parent.sc private def appName = parent.appName diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index c9897d4fa4ca9..45e647da9968a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[spark] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { private val poolToActiveStages = listener.poolToActiveStages private def listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 37e7e6302599f..81ea66f639495 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,7 +28,7 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[spark] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressUI) { private val dateFmt = parent.dateFmt private def appName = parent.appName private def listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 3983bb00184d5..7785c6fffd5ee 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,7 +27,7 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { private val dateFmt = parent.dateFmt private def isFairScheduler = parent.isFairScheduler private def listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a9c2d956f0b6a..cee7426b9a9e5 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -21,14 +21,11 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import org.apache.spark.SparkContext -import org.apache.spark.scheduler._ -import org.apache.spark.storage.{StorageUtils, RDDInfo} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(parent: SparkUI) { +private[ui] class BlockManagerUI(parent: SparkUI) { val live = parent.live val sc = parent.sc @@ -41,7 +38,7 @@ private[spark] class BlockManagerUI(parent: SparkUI) { def start() { val gateway = parent.gatewayListener - _listener = Some(new BlockManagerListener(sc, gateway, live)) + _listener = Some(new BlockManagerListener) gateway.registerSparkListener(listener) } @@ -54,32 +51,4 @@ private[spark] class BlockManagerUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[spark] class BlockManagerListener( - sc: SparkContext, - gateway: GatewayUISparkListener, - live: Boolean) - extends StorageStatusFetchSparkListener(sc, gateway, live) { - var rddInfoList: Seq[RDDInfo] = Seq() - - def getRDDInfo() { - if (live) { - val rddInfo = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val getRDDInfo = new SparkListenerGetRDDInfo(rddInfo) - gateway.onGetRDDInfo(getRDDInfo) - } - } - - override def onGetRDDInfo(getRDDInfo: SparkListenerGetRDDInfo) { - rddInfoList = getRDDInfo.rddInfoList - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { - super.onStageSubmitted(stageSubmitted) - getRDDInfo() - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = { - super.onStageCompleted(stageCompleted) - getRDDInfo() - } -} +private[ui] class BlockManagerListener extends RDDInfoSparkListener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 3c4cb119fc05b..f24bdf366d94b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -27,14 +27,12 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[spark] class IndexPage(parent: BlockManagerUI) { +private[ui] class IndexPage(parent: BlockManagerUI) { private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { // Calculate macro-level statistics - listener.fetchStorageStatus() - listener.getRDDInfo() val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, appName, "Storage ", Storage) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 0f8ac552ab052..290520a015fa0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,36 +21,31 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils} -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus +import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[spark] class RDDPage(parent: BlockManagerUI) { +private[ui] class RDDPage(parent: BlockManagerUI) { private def appName = parent.appName private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - listener.fetchStorageStatus() - listener.getRDDInfo() val storageStatusList = listener.storageStatusList val id = request.getParameter("id").toInt - val filteredStorageStatusList = - StorageUtils.filterStorageStatusByRDD(storageStatusList.toArray, id) val rddInfo = listener.rddInfoList.filter(_.id == id).head // Worker table - val workers = filteredStorageStatusList.map((id, _)) + val workers = storageStatusList.map((id, _)) val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray. - sortWith(_._1.name < _._1.name) + val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) - val blocks = blockStatuses.map { case (id, status) => - (id, status, blockLocations.get(id).getOrElse(Seq("Unknown"))) + val blocks = blockStatuses.map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq("Unknown"))) } val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) 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 f3af976cf1098..5a26a77bee728 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,18 +21,20 @@ import java.util.{UUID, Properties} import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.mutable + +import net.liftweb.json.JsonDSL._ +import net.liftweb.json.JsonAST._ +import net.liftweb.json.DefaultFormats + import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.storage._ import org.apache.spark._ -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats - private[spark] object JsonProtocol { + private implicit val format = DefaultFormats /** * JSON serialization methods for SparkListenerEvent's @@ -41,30 +43,31 @@ private[spark] object JsonProtocol { def sparkEventToJson(event: SparkListenerEvent): JValue = { event match { case stageSubmitted: SparkListenerStageSubmitted => - stageSubmittedEventToJson(stageSubmitted) + stageSubmittedToJson(stageSubmitted) case stageCompleted: SparkListenerStageCompleted => - stageCompletedEventToJson(stageCompleted) + stageCompletedToJson(stageCompleted) case taskStart: SparkListenerTaskStart => - taskStartEventToJson(taskStart) + taskStartToJson(taskStart) case taskGettingResult: SparkListenerTaskGettingResult => - taskGettingResultEventToJson(taskGettingResult) + taskGettingResultToJson(taskGettingResult) case taskEnd: SparkListenerTaskEnd => - taskEndEventToJson(taskEnd) + taskEndToJson(taskEnd) case jobStart: SparkListenerJobStart => - jobStartEventToJson(jobStart) + jobStartToJson(jobStart) case jobEnd: SparkListenerJobEnd => - jobEndEventToJson(jobEnd) + jobEndToJson(jobEnd) case applicationStart: SparkListenerApplicationStart => - applicationStartEventToJson(applicationStart) - case storageStatusFetch: SparkListenerStorageStatusFetch => - storageStatusFetchEventToJson(storageStatusFetch) - case getRDDInfo: SparkListenerGetRDDInfo => - getRDDInfoEventToJson(getRDDInfo) - // SparkListenerShutdown is not supported + applicationStartToJson(applicationStart) + case environmentUpdate: SparkListenerEnvironmentUpdate => + environmentUpdateToJson(environmentUpdate) + case executorsStateChange: SparkListenerExecutorsStateChange => + executorsStateChangeToJson(executorsStateChange) + case SparkListenerShutdown => + shutdownToJson() } } - def stageSubmittedEventToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { + def stageSubmittedToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) val properties = propertiesToJson(stageSubmitted.properties) ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ @@ -72,26 +75,26 @@ private[spark] object JsonProtocol { ("Properties" -> properties) } - def stageCompletedEventToJson(stageCompleted: SparkListenerStageCompleted): JValue = { + def stageCompletedToJson(stageCompleted: SparkListenerStageCompleted): JValue = { val stageInfo = stageInfoToJson(stageCompleted.stageInfo) ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ ("Stage Info" -> stageInfo) } - def taskStartEventToJson(taskStart: SparkListenerTaskStart): JValue = { + def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskInfoToJson(taskStart.taskInfo) ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ ("Task Info" -> taskInfo) } - def taskGettingResultEventToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { + def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskInfoToJson(taskGettingResult.taskInfo) ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ ("Task Info" -> taskInfo) } - def taskEndEventToJson(taskEnd: SparkListenerTaskEnd): JValue = { + def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskInfoToJson(taskEnd.taskInfo) val taskMetrics = taskMetricsToJson(taskEnd.taskMetrics) @@ -103,7 +106,7 @@ private[spark] object JsonProtocol { ("Task Metrics" -> taskMetrics) } - def jobStartEventToJson(jobStart: SparkListenerJobStart): JValue = { + def jobStartToJson(jobStart: SparkListenerJobStart): JValue = { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ @@ -111,37 +114,41 @@ private[spark] object JsonProtocol { ("Properties" -> properties) } - def jobEndEventToJson(jobEnd: SparkListenerJobEnd): JValue = { + def jobEndToJson(jobEnd: SparkListenerJobEnd): JValue = { val jobResult = jobResultToJson(jobEnd.jobResult) ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ ("Job ID" -> jobEnd.jobId) ~ ("Job Result" -> jobResult) } - def applicationStartEventToJson(applicationStart: SparkListenerApplicationStart): JValue = { - val environmentDetails = applicationStart.environmentDetails + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) + } + + def environmentUpdateToJson(environmentUpdate: SparkListenerEnvironmentUpdate): JValue = { + val environmentDetails = environmentUpdate.environmentDetails val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) val systemProperties = mapToJson(environmentDetails("System Properties").toMap) val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) - ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("Event" -> Utils.getFormattedClassName(environmentUpdate)) ~ ("JVM Information" -> jvmInformation) ~ ("Spark Properties" -> sparkProperties) ~ ("System Properties" -> systemProperties) ~ ("Classpath Entries" -> classpathEntries) } - def storageStatusFetchEventToJson(storageStatusFetch: SparkListenerStorageStatusFetch): JValue = { + def executorsStateChangeToJson(executorsStateChange: SparkListenerExecutorsStateChange) + : JValue = { val storageStatusList = - JArray(storageStatusFetch.storageStatusList.map(storageStatusToJson).toList) - ("Event" -> Utils.getFormattedClassName(storageStatusFetch)) ~ - ("Storage Status List" -> storageStatusList) + JArray(executorsStateChange.storageStatusList.map(storageStatusToJson).toList) + ("Event" -> Utils.getFormattedClassName(executorsStateChange)) ~ + ("Storage Status List" -> storageStatusList) } - def getRDDInfoEventToJson(getRDDInfo: SparkListenerGetRDDInfo): JValue = { - val rddInfoList = JArray(getRDDInfo.rddInfoList.map(rddInfoToJson).toList) - ("Event" -> Utils.getFormattedClassName(getRDDInfo)) ~ - ("RDD Info List" -> rddInfoList) + def shutdownToJson(): JValue = { + "Event" -> Utils.getFormattedClassName(SparkListenerShutdown) } /** @@ -149,6 +156,7 @@ private[spark] object JsonProtocol { */ def stageInfoToJson(stageInfo: StageInfo): JValue = { + val rddInfo = rddInfoToJson(stageInfo.rddInfo) val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => ("Task Info" -> taskInfoToJson(info)) ~ ("Task Metrics" -> taskMetricsToJson(metrics)) @@ -157,9 +165,8 @@ private[spark] object JsonProtocol { val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ - ("RDD Name" -> stageInfo.rddName) ~ - ("Number of Partitions" -> stageInfo.numPartitions) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ + ("RDD Info" -> rddInfo) ~ ("Task Infos" -> taskInfos) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ @@ -184,6 +191,12 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) + val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + JArray(blocks.toList.map { case (id, status) => + ("Block ID" -> blockIdToJson(id)) ~ + ("Status" -> blockStatusToJson(status)) + }) + }.getOrElse(JNothing) ("Host Name" -> taskMetrics.hostname) ~ ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ ("Executor Run Time" -> taskMetrics.executorRunTime) ~ @@ -193,7 +206,8 @@ private[spark] object JsonProtocol { ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ - ("Shuffle Write Metrics" -> shuffleWriteMetrics) + ("Shuffle Write Metrics" -> shuffleWriteMetrics) ~ + ("Updated Blocks" -> updatedBlocks) } def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { @@ -268,8 +282,8 @@ private[spark] object JsonProtocol { ("RDD ID" -> rddInfo.id) ~ ("Name" -> rddInfo.name) ~ ("Storage Level" -> storageLevel) ~ - ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Number of Partitions" -> rddInfo.numPartitions) ~ + ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ ("Disk Size" -> rddInfo.diskSize) } @@ -364,11 +378,10 @@ private[spark] object JsonProtocol { val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) - val storageStatusFetch = Utils.getFormattedClassName(SparkListenerStorageStatusFetch) - val getRDDInfo = Utils.getFormattedClassName(SparkListenerGetRDDInfo) - // SparkListenerShutdown is not supported + val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + val executorsStateChanged = Utils.getFormattedClassName(SparkListenerExecutorsStateChange) + val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) - implicit val format = DefaultFormats (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) case `stageCompleted` => stageCompletedFromJson(json) @@ -378,8 +391,9 @@ private[spark] object JsonProtocol { case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) case `applicationStart` => applicationStartFromJson(json) - case `storageStatusFetch` => storageStatusFetchFromJson(json) - case `getRDDInfo` => getRDDInfoFromJson(json) + case `environmentUpdate` => environmentUpdateFromJson(json) + case `executorsStateChanged` => executorsStateChangeFromJson(json) + case `shutdown` => SparkListenerShutdown } } @@ -395,7 +409,6 @@ private[spark] object JsonProtocol { } def taskStartFromJson(json: JValue): SparkListenerTaskStart = { - implicit val format = DefaultFormats val stageId = (json \ "Stage ID").extract[Int] val taskInfo = taskInfoFromJson(json \ "Task Info") new SparkListenerTaskStart(stageId, taskInfo) @@ -407,7 +420,6 @@ private[spark] object JsonProtocol { } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { - implicit val format = DefaultFormats val stageId = (json \ "Stage ID").extract[Int] val taskType = (json \ "Task Type").extract[String] val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") @@ -417,7 +429,6 @@ private[spark] object JsonProtocol { } def jobStartFromJson(json: JValue): SparkListenerJobStart = { - implicit val format = DefaultFormats val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") @@ -425,32 +436,28 @@ private[spark] object JsonProtocol { } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { - implicit val format = DefaultFormats val jobId = (json \ "Job ID").extract[Int] val jobResult = jobResultFromJson(json \ "Job Result") new SparkListenerJobEnd(jobId, jobResult) } def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + new SparkListenerApplicationStart((json \ "App Name").extract[String]) + } + + def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { val environmentDetails = Map[String, Seq[(String, String)]]( "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, "System Properties" -> mapFromJson(json \ "System Properties").toSeq, "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) - new SparkListenerApplicationStart(environmentDetails) + new SparkListenerEnvironmentUpdate(environmentDetails) } - def storageStatusFetchFromJson(json: JValue): SparkListenerStorageStatusFetch = { - implicit val format = DefaultFormats + def executorsStateChangeFromJson(json: JValue): SparkListenerExecutorsStateChange = { val storageStatusList = (json \ "Storage Status List").extract[List[JValue]].map(storageStatusFromJson) - new SparkListenerStorageStatusFetch(storageStatusList) - } - - def getRDDInfoFromJson(json: JValue): SparkListenerGetRDDInfo = { - implicit val format = DefaultFormats - val rddInfoList = (json \ "RDD Info List").extract[List[JValue]].map(rddInfoFromJson) - new SparkListenerGetRDDInfo(rddInfoList) + new SparkListenerExecutorsStateChange(storageStatusList) } /** @@ -458,12 +465,10 @@ private[spark] object JsonProtocol { */ def stageInfoFromJson(json: JValue): StageInfo = { - implicit val format = DefaultFormats val stageId = (json \ "Stage ID").extract[Int] val stageName = (json \ "Stage Name").extract[String] - val rddName = (json \ "RDD Name").extract[String] - val numPartitions = (json \ "Number of Partitions").extract[Int] val numTasks = (json \ "Number of Tasks").extract[Int] + val rddInfo = rddInfoFromJson(json \ "RDD Info") val taskInfos = (json \ "Task Infos").extract[List[JValue]].map { value => (taskInfoFromJson(value \ "Task Info"), taskMetricsFromJson(value \ "Task Metrics")) }.toBuffer @@ -471,7 +476,7 @@ private[spark] object JsonProtocol { val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, rddName, numPartitions, numTasks, taskInfos) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo, taskInfos) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning @@ -479,7 +484,6 @@ private[spark] object JsonProtocol { } def taskInfoFromJson(json: JValue): TaskInfo = { - implicit val format = DefaultFormats val taskId = (json \ "Task ID").extract[Long] val index = (json \ "Index").extract[Int] val launchTime = (json \ "Launch Time").extract[Long] @@ -500,7 +504,6 @@ private[spark] object JsonProtocol { } def taskMetricsFromJson(json: JValue): TaskMetrics = { - implicit val format = DefaultFormats val metrics = new TaskMetrics metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] @@ -513,11 +516,17 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = blockIdFromJson(block \ "Block ID") + val status = blockStatusFromJson(block \ "Status") + (id, status) + } + } metrics } def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { - implicit val format = DefaultFormats val metrics = new ShuffleReadMetrics metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] @@ -530,7 +539,6 @@ private[spark] object JsonProtocol { } def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { - implicit val format = DefaultFormats val metrics = new ShuffleWriteMetrics metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] @@ -538,7 +546,6 @@ private[spark] object JsonProtocol { } def taskEndReasonFromJson(json: JValue): TaskEndReason = { - implicit val format = DefaultFormats val success = Utils.getFormattedClassName(Success) val resubmitted = Utils.getFormattedClassName(Resubmitted) val fetchFailed = Utils.getFormattedClassName(FetchFailed) @@ -571,7 +578,6 @@ private[spark] object JsonProtocol { } def blockManagerIdFromJson(json: JValue): BlockManagerId = { - implicit val format = DefaultFormats val executorId = (json \ "Executor ID").extract[String] val host = (json \ "Host").extract[String] val port = (json \ "Port").extract[Int] @@ -580,7 +586,6 @@ private[spark] object JsonProtocol { } def jobResultFromJson(json: JValue): JobResult = { - implicit val format = DefaultFormats val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) val jobFailed = Utils.getFormattedClassName(JobFailed) @@ -594,31 +599,34 @@ private[spark] object JsonProtocol { } def storageStatusFromJson(json: JValue): StorageStatus = { - implicit val format = DefaultFormats val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] val blocks = (json \ "Blocks").extract[List[JValue]].map { block => val id = blockIdFromJson(block \ "Block ID") val status = blockStatusFromJson(block \ "Status") (id, status) - }.toMap - new StorageStatus(blockManagerId, maxMem, blocks) + } + val blockMap = mutable.Map[BlockId, BlockStatus](blocks: _*) + new StorageStatus(blockManagerId, maxMem, blockMap) } def rddInfoFromJson(json: JValue): RDDInfo = { - implicit val format = DefaultFormats val rddId = (json \ "RDD ID").extract[Int] val name = (json \ "Name").extract[String] val storageLevel = storageLevelFromJson(json \ "Storage Level") - val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val numPartitions = (json \ "Number of Partitions").extract[Int] + val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - new RDDInfo(rddId, name, storageLevel, numCachedPartitions, numPartitions, memSize, diskSize) + + val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) + rddInfo.numCachedPartitions = numCachedPartitions + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo } def storageLevelFromJson(json: JValue): StorageLevel = { - implicit val format = DefaultFormats val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] val deserialized = (json \ "Deserialized").extract[Boolean] @@ -627,7 +635,6 @@ private[spark] object JsonProtocol { } def blockIdFromJson(json: JValue): BlockId = { - implicit val format = DefaultFormats val rddBlockId = Utils.getFormattedClassName(RDDBlockId) val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) @@ -672,11 +679,10 @@ private[spark] object JsonProtocol { } def blockStatusFromJson(json: JValue): BlockStatus = { - implicit val format = DefaultFormats val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - new BlockStatus(storageLevel, memorySize, diskSize) + BlockStatus(storageLevel, memorySize, diskSize) } /** @@ -697,14 +703,12 @@ private[spark] object JsonProtocol { } def UUIDFromJson(json: JValue): UUID = { - implicit val format = DefaultFormats val leastSignificantBits = (json \ "Least Significant Bits").extract[Long] val mostSignificantBits = (json \ "Most Significant Bits").extract[Long] new UUID(leastSignificantBits, mostSignificantBits) } def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { - implicit val format = DefaultFormats json.extract[List[JValue]].map { line => val declaringClass = (line \ "Declaring Class").extract[String] val methodName = (line \ "Method Name").extract[String] @@ -715,7 +719,6 @@ private[spark] object JsonProtocol { } def exceptionFromJson(json: JValue): Exception = { - implicit val format = DefaultFormats val e = new Exception((json \ "Message").extract[String]) e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) e diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index a34976af44963..d88343de75cb1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -50,9 +50,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val first = listener.stageInfos.head - first.rddName should be {"Target RDD"} + first.rddInfo.name should be {"Target RDD"} first.numTasks should be {4} - first.numPartitions should be {4} + first.rddInfo.numPartitions should be {4} first.submissionTime should be ('defined) first.completionTime should be ('defined) first.taskInfos.length should be {4} @@ -110,7 +110,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc checkNonZeroAvg( stageInfo.taskInfos.map{_._2.executorDeserializeTime}, stageInfo + " executorDeserializeTime") - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { checkNonZeroAvg( stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime") @@ -118,11 +118,11 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) { + if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) 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 9483fd8fb10f2..006353b705e32 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 @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc) + val listener = new JobProgressListener(sc, true) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() From d1f428591d6c33c2bb86f85468c7842b5ca00311 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 26 Feb 2014 17:19:20 -0800 Subject: [PATCH 30/68] Migrate from lift-json to json4s-jackson --- .../apache/spark/deploy/master/ui/ApplicationPage.scala | 1 - .../org/apache/spark/deploy/master/ui/IndexPage.scala | 1 - core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../main/scala/org/apache/spark/ui/UISparkListener.scala | 6 +++--- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 7 +++---- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 6 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 90cad3c37fda6..31a431078698b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index bac922bcd3d98..e3286606ced48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} 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 6ad25b92161b7..594dce8eac9f3 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -22,8 +22,8 @@ import java.io.{FileInputStream, File} import scala.io.Source import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import net.liftweb.json._ import org.eclipse.jetty.server.{Handler, Server} +import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 25d8c3d5d001a..03d419eebbcdc 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import net.liftweb.json.JsonAST._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ @@ -54,8 +54,8 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends /** Log the event as JSON */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { - val eventJson = JsonProtocol.sparkEventToJson(event) - logger.foreach(_.logLine(compactRender(eventJson))) + val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) + logger.foreach(_.logLine(eventJson)) if (flushLogger) { logger.foreach(_.flush()) } 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 5a26a77bee728..d731033d9f96a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -23,10 +23,9 @@ import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable -import net.liftweb.json.JsonDSL._ -import net.liftweb.json.JsonAST._ -import net.liftweb.json.DefaultFormats - +import org.json4s.JsonDSL._ +import org.json4s.JsonAST._ +import org.json4s.DefaultFormats import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 72a8ec156b51d..55eee311a5da1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -31,7 +31,7 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import net.liftweb.json.JsonAST._ +import org.json4s._ import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.apache.spark.{Logging, SparkConf, SparkException} From 7b2f8112795a53c35b10bc3d72e5be7b699ceb65 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 11:24:32 -0800 Subject: [PATCH 31/68] Guard against TaskMetrics NPE + Fix tests --- .../main/scala/org/apache/spark/CacheManager.scala | 6 ++++-- .../scala/org/apache/spark/scheduler/JobLogger.scala | 3 ++- .../scala/org/apache/spark/ui/UISparkListener.scala | 12 ++++++++---- .../apache/spark/ui/jobs/JobProgressListener.scala | 3 ++- .../scala/org/apache/spark/util/JsonProtocol.scala | 8 +++++--- .../scala/org/apache/spark/CacheManagerSuite.scala | 5 +++-- .../apache/spark/scheduler/SparkListenerSuite.scala | 4 ++-- 7 files changed, 26 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 35a9956716e64..0876830fe0586 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -77,9 +77,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { elements ++= computedValues val updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) - // Update task metrics to include any updated blocks + // Update task metrics to include any blocks whose storage status is updated val metrics = context.taskMetrics - metrics.updatedBlocks = Some(updatedBlocks ++ metrics.updatedBlocks.getOrElse(Seq())) + if (metrics != null) { + metrics.updatedBlocks = Some(updatedBlocks ++ metrics.updatedBlocks.getOrElse(Seq())) + } elements.iterator.asInstanceOf[Iterator[T]] } finally { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 7faa7eab13310..a7d2342e2d079 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -195,9 +195,10 @@ class JobLogger(val user: String, val logDirName: String) override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) + recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) case Resubmitted => taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 03d419eebbcdc..4ece4ac6bc345 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -135,9 +135,12 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val execId = taskEnd.taskInfo.executorId - val updatedBlocks = taskEnd.taskMetrics.updatedBlocks.getOrElse(Seq()) - if (updatedBlocks.length > 0) { - updateStorageStatus(execId, updatedBlocks) + val metrics = taskEnd.taskMetrics + if (metrics != null) { + val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq()) + if (updatedBlocks.length > 0) { + updateStorageStatus(execId, updatedBlocks) + } } } @@ -163,7 +166,8 @@ private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { super.onTaskEnd(taskEnd) - if (taskEnd.taskMetrics.updatedBlocks.isDefined) { + val metrics = taskEnd.taskMetrics + if (metrics != null && metrics.updatedBlocks.isDefined) { updateRDDInfo() } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 495ea62d75739..54193b5057ded 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -159,7 +159,8 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) // update duration y.taskTime += taskEnd.taskInfo.duration - Option(taskEnd.taskMetrics).foreach { taskMetrics => + val taskMetrics = taskEnd.taskMetrics + if (taskMetrics != null) { taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } y.memoryBytesSpilled += taskMetrics.memoryBytesSpilled 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 d731033d9f96a..1520c0d2bc716 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -96,13 +96,14 @@ private[spark] object JsonProtocol { def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskInfoToJson(taskEnd.taskInfo) - val taskMetrics = taskMetricsToJson(taskEnd.taskMetrics) + val taskMetrics = taskEnd.taskMetrics + val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ ("Task Info" -> taskInfo) ~ - ("Task Metrics" -> taskMetrics) + ("Task Metrics" -> taskMetricsJson) } def jobStartToJson(jobStart: SparkListenerJobStart): JValue = { @@ -157,8 +158,9 @@ private[spark] object JsonProtocol { def stageInfoToJson(stageInfo: StageInfo): JValue = { val rddInfo = rddInfoToJson(stageInfo.rddInfo) val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => + val metricsJson = if (metrics != null) taskMetricsToJson(metrics) else JNothing ("Task Info" -> taskInfoToJson(info)) ~ - ("Task Metrics" -> taskMetricsToJson(metrics)) + ("Task Metrics" -> metricsJson) }.toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index ea936e815b24a..77ecea4f709f5 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} +import org.apache.spark.executor.TaskMetrics // TODO: Test the CacheManager's thread-safety aspects class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { @@ -54,12 +55,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar expecting { blockManager.get(RDDBlockId(0, 0)).andReturn(None) blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andReturn(0) + true).andStubReturn(Seq()) } whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index d88343de75cb1..a4c38d94e041c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -70,7 +70,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val first = listener.stageInfos.head first.numTasks should be {2} - first.numPartitions should be {4} + first.rddInfo.numPartitions should be {4} } test("local metrics") { @@ -164,7 +164,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) - assert(listener.startedGettingResultTasks.isEmpty == true) + assert(listener.startedGettingResultTasks.isEmpty) assert(listener.endedTasks.contains(TASK_INDEX)) } From 996d7a2f42d4e02c1e40ec22b0c4d7db86aa03e3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 12:26:23 -0800 Subject: [PATCH 32/68] Reflect RDD unpersist on UI This introduces a new event, SparkListenerUnpersistRDD. --- .../scala/org/apache/spark/SparkContext.scala | 19 +++++++++++- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 ++-- .../spark/scheduler/SparkListener.scala | 6 ++++ .../spark/scheduler/SparkListenerBus.scala | 2 ++ .../org/apache/spark/ui/UISparkListener.scala | 31 +++++++++++++++++-- .../org/apache/spark/util/JsonProtocol.scala | 13 ++++++++ 6 files changed, 69 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 32e43097ee3a9..66dc7f4484a73 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -700,7 +700,7 @@ class SparkContext( } /** - * Return current scheduling mode + * Return current scheduling mode */ def getSchedulingMode: SchedulingMode.SchedulingMode = { taskScheduler.schedulingMode @@ -727,6 +727,23 @@ class SparkContext( dagScheduler.getPreferredLocs(rdd, partition) } + /** + * Register an RDD to be persisted in memory and/or disk storage + */ + private[spark] def persistRDD(rdd: RDD[_]) { + persistentRdds(rdd.id) = rdd + } + + /** + * Unpersist an RDD from memory and/or disk storage + */ + private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { + val rddId = rdd.id + env.blockManager.master.removeRdd(rddId, blocking) + persistentRdds.remove(rddId) + dagScheduler.post(new SparkListenerUnpersistRDD(rddId)) + } + /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 50320f40350cd..a096aa505986c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -145,9 +145,8 @@ abstract class RDD[T: ClassTag]( throw new UnsupportedOperationException( "Cannot change storage level of an RDD after it was already assigned a level") } + sc.persistRDD(this) storageLevel = newLevel - // Register the RDD with the SparkContext - sc.persistentRdds(id) = this this } @@ -165,8 +164,7 @@ abstract class RDD[T: ClassTag]( */ def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id, blocking) - sc.persistentRdds.remove(id) + sc.unpersistRDD(this, blocking) storageLevel = StorageLevel.NONE this } 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 6ae05e75b6387..2e3d69f2f4e9c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -53,6 +53,8 @@ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(S case class SparkListenerExecutorsStateChange(storageStatusList: Seq[StorageStatus]) extends SparkListenerEvent +case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -112,6 +114,10 @@ trait SparkListener { */ def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { } + /** + * Called when an RDD is manually unpersisted by the application + */ + def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 3a7d0ff9408a7..306f5fc5b610c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -75,6 +75,8 @@ private[spark] class SparkListenerBus extends Logging { listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) case executorsStateChange: SparkListenerExecutorsStateChange => listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) + case unpersistRDD: SparkListenerUnpersistRDD => + listeners.foreach(_.onUnpersistRDD(unpersistRDD)) case SparkListenerShutdown => return true case _ => diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 4ece4ac6bc345..02f42f0b3592e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -61,8 +61,8 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends } } + private def startLogger() = logger.foreach(_.start()) private def closeLogger() = logger.foreach(_.close()) - private def restartLogger() = logger.foreach(_.start()) override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { listeners.foreach(_.onStageSubmitted(stageSubmitted)) @@ -89,7 +89,7 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends override def onJobStart(jobStart: SparkListenerJobStart) { listeners.foreach(_.onJobStart(jobStart)) - restartLogger() + startLogger() logEvent(jobStart) } @@ -115,6 +115,13 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) logEvent(executorsStateChange, flushLogger = true) } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { + listeners.foreach(_.onUnpersistRDD(unpersistRDD)) + // In case logger has not already started, as unpersist may be called between jobs + startLogger() + logEvent(unpersistRDD, flushLogger = true) + } } /** @@ -133,6 +140,16 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } } + /** Update storage status list to reflect the removal of an RDD from the cache */ + def updateStorageStatus(unpersistedRDDId: Int) { + storageStatusList.foreach { storageStatus => + val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) + unpersistedBlocksIds.foreach { blockId => + storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L) + } + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val execId = taskEnd.taskInfo.executorId val metrics = taskEnd.taskMetrics @@ -144,6 +161,10 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } } + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { + updateStorageStatus(unpersistRDD.rddId) + } + override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { storageStatusList = executorsStateChange.storageStatusList } @@ -179,7 +200,11 @@ private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { // Remove all partitions that are no longer cached - // TODO(aor): Handle unpersist _rddInfoMap.retain { case (id, info) => info.numCachedPartitions > 0 } } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { + super.onUnpersistRDD(unpersistRDD) + updateRDDInfo() + } } 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 1520c0d2bc716..e10b41a62e121 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -61,6 +61,8 @@ private[spark] object JsonProtocol { environmentUpdateToJson(environmentUpdate) case executorsStateChange: SparkListenerExecutorsStateChange => executorsStateChangeToJson(executorsStateChange) + case unpersistRDD: SparkListenerUnpersistRDD => + unpersistRDDToJson(unpersistRDD) case SparkListenerShutdown => shutdownToJson() } @@ -147,6 +149,11 @@ private[spark] object JsonProtocol { ("Storage Status List" -> storageStatusList) } + def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { + ("Event" -> Utils.getFormattedClassName(unpersistRDD)) ~ + ("RDD ID" -> unpersistRDD.rddId) + } + def shutdownToJson(): JValue = { "Event" -> Utils.getFormattedClassName(SparkListenerShutdown) } @@ -381,6 +388,7 @@ private[spark] object JsonProtocol { val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) val executorsStateChanged = Utils.getFormattedClassName(SparkListenerExecutorsStateChange) + val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) (json \ "Event").extract[String] match { @@ -394,6 +402,7 @@ private[spark] object JsonProtocol { case `applicationStart` => applicationStartFromJson(json) case `environmentUpdate` => environmentUpdateFromJson(json) case `executorsStateChanged` => executorsStateChangeFromJson(json) + case `unpersistRDD` => unpersistRDDFromJson(json) case `shutdown` => SparkListenerShutdown } } @@ -461,6 +470,10 @@ private[spark] object JsonProtocol { new SparkListenerExecutorsStateChange(storageStatusList) } + def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { + new SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) + } + /** * JSON deserialization methods for classes SparkListenerEvent's depend on */ From 472fd8a4845e39a38f8d993a3527a7e77571ffad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 15:03:59 -0800 Subject: [PATCH 33/68] Fix a couple of tests --- .../apache/spark/scheduler/DAGScheduler.scala | 7 +++++-- .../spark/scheduler/SparkListener.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 21 +++++++++---------- .../spark/scheduler/JobLoggerSuite.scala | 3 ++- 4 files changed, 18 insertions(+), 15 deletions(-) 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 ef2b2167a9734..81c88b0f9b40c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -987,8 +987,11 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } - val storageStatusList = blockManagerMaster.getStorageStatus - post(new SparkListenerExecutorsStateChange(storageStatusList)) + // Block manager master actor should not be null except during tests + if (blockManagerMaster.driverActor != null) { + val storageStatusList = blockManagerMaster.getStorageStatus + post(new SparkListenerExecutorsStateChange(storageStatusList)) + } } private def handleExecutorGained(execId: String, host: String) { 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 2e3d69f2f4e9c..56d7a012f5068 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -28,7 +28,7 @@ import org.apache.spark.storage.StorageStatus sealed trait SparkListenerEvent -case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties) +case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e49b6aecd78c7..393f49df9bd9a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -631,17 +631,16 @@ private[spark] class BlockManager( reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } updatedBlocks += ((blockId, putBlockStatus)) - } catch { - case e: Exception => - // If we failed in putting the block to memory/disk, notify other possible readers - // that it has failed, and then remove it from the block info map. - if (!marked) { - // Note that the remove must happen before markFailure otherwise another thread - // could've inserted a new BlockInfo before we remove it. - blockInfo.remove(blockId) - putBlockInfo.markFailure() - logWarning("Putting block " + blockId + " failed") - } + } finally { + // If we failed in putting the block to memory/disk, notify other possible readers + // that it has failed, and then remove it from the block info map. + if (!marked) { + // Note that the remove must happen before markFailure otherwise another thread + // could've inserted a new BlockInfo before we remove it. + blockInfo.remove(blockId) + putBlockInfo.markFailure() + logWarning("Putting block " + blockId + " failed") + } } } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index b13dc1c56ad6d..d0c22cc937a16 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -54,7 +54,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None) val rootStageInfo = StageInfo.fromStage(rootStage) - joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) + joblogger.onJobStart(SparkListenerJobStart(jobID, Seq[Int](0, 1))) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo)) joblogger.createLogWriterTest(jobID) joblogger.getJobIDToPrintWriter.size should be (1) joblogger.getJobIDToStageIDs.get(jobID).get.size should be (2) From d47585f22f243fc7e840af90132edb7e84b003ed Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 16:15:21 -0800 Subject: [PATCH 34/68] Clean up FileLogger --- .../org/apache/spark/util/FileLogger.scala | 37 ++++++------------- 1 file changed, 11 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a483489200700..b71232d920c6c 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -25,37 +25,27 @@ import org.apache.spark.Logging /** * A generic class for logging information to file - * @param user User identifier if SPARK_LOG_DIR is not set, in which case log directory - * defaults to /tmp/spark-[user] - * @param name Name of logger, also the base name of the log files - * @param flushPeriod How many writes until the results are flushed to disk. By default, - * only flush manually + * @param logDir Path to the directory in which files are logged + * @param name An identifier of each FileLogger instance */ -class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALUE) extends Logging { +class FileLogger( + logDir: String = Option(System.getenv("SPARK_LOG_DIR")) + .getOrElse("/tmp/spark-%s".format(System.getProperty("user.name", "user"))), + name: String = String.valueOf(System.currentTimeMillis())) + extends Logging { + + private val logPath = logDir.stripSuffix("/") + "/" + name private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private var logCount = 0 private var fileIndex = 0 - private val logDir = - if (System.getenv("SPARK_LOG_DIR") != null) { - "%s/%s/".format(System.getenv("SPARK_LOG_DIR"), name) - } else { - "/tmp/spark-%s/%s/".format(user, name) - } - private var writer: Option[PrintWriter] = { createLogDir() Some(createWriter()) } - def this() = this(System.getProperty("user.name", ""), - String.valueOf(System.currentTimeMillis())) - - def this(_name: String) = this(System.getProperty("user.name", ""), _name) - /** Create a logging directory with the given path */ private def createLogDir() = { - val dir = new File(logDir) + val dir = new File(logPath) if (dir.exists) { logWarning("Logging directory already exists: " + logDir) } @@ -68,7 +58,7 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU /** Create a new writer to the file identified with the given path */ private def createWriter() = { // Overwrite any existing file - val fileWriter = new FileWriter(logDir + fileIndex) + val fileWriter = new FileWriter(logPath + "/" + fileIndex) val bufferedWriter = new BufferedWriter(fileWriter) new PrintWriter(bufferedWriter) } @@ -85,11 +75,6 @@ class FileLogger(user: String, name: String, flushPeriod: Int = Integer.MAX_VALU writeInfo = DATE_FORMAT.format(date) + ": " + msg } writer.foreach(_.print(writeInfo)) - logCount += 1 - if (logCount % flushPeriod == 0) { - flush() - logCount = 0 - } } /** From faa113e674a276ddf5cd7dc643c16b7bed2b5e44 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 17:12:19 -0800 Subject: [PATCH 35/68] General clean up --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- .../main/scala/org/apache/spark/storage/BlockManager.scala | 6 +++--- .../spark/storage/BlockManagerRegistrationListener.scala | 2 +- .../main/scala/org/apache/spark/ui/env/EnvironmentUI.scala | 5 ++--- .../main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala | 5 ++--- .../main/scala/org/apache/spark/ui/jobs/IndexPage.scala | 6 +++--- .../scala/org/apache/spark/ui/jobs/JobProgressUI.scala | 7 +++---- .../src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala | 4 ++-- .../main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- .../scala/org/apache/spark/ui/storage/BlockManagerUI.scala | 5 ++--- .../main/scala/org/apache/spark/ui/storage/IndexPage.scala | 4 ++-- .../main/scala/org/apache/spark/ui/storage/RDDPage.scala | 4 ++-- 12 files changed, 25 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 66dc7f4484a73..91ced298a48d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1056,7 +1056,7 @@ class SparkContext( SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) val environmentUpdate = new SparkListenerEnvironmentUpdate(environmentDetails) - // In case the DAG scheduler is not ready yet, first check whether its reference is valid + // DAG scheduler may not be ready yet Option(dagScheduler).foreach(_.post(environmentUpdate)) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 393f49df9bd9a..dd77b5394e16c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -236,10 +236,10 @@ private[spark] class BlockManager( info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - val storageLevel = status.storageLevel - val inMemSize = Math.max(status.memSize, droppedMemorySize) - val onDiskSize = status.diskSize if (info.tellMaster) { + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) } else { true diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala index 1825198c7741f..31ad13cacdb79 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala @@ -21,7 +21,7 @@ import org.apache.spark.scheduler._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable -/** A listener for block manager state changes */ +/** A listener for block manager registration */ private[spark] class BlockManagerRegistrationListener { private var _listenerBus: Option[SparkListenerBus] = None diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 798c13f65b862..c536fde3efc4c 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -29,14 +29,13 @@ import org.apache.spark.ui.Page.Environment import org.apache.spark.ui._ private[ui] class EnvironmentUI(parent: SparkUI) { + lazy val appName = parent.appName + lazy val listener = _listener.get val live = parent.live val sc = parent.sc private var _listener: Option[EnvironmentListener] = None - def appName = parent.appName - def listener = _listener.get - def start() { val gateway = parent.gatewayListener _listener = Some(new EnvironmentListener()) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 58f79e1435e75..0da815c402f87 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -32,14 +32,13 @@ import org.apache.spark.ui._ import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { + lazy val appName = parent.appName + lazy val listener = _listener.get val live = parent.live val sc = parent.sc private var _listener: Option[ExecutorsListener] = None - def appName = parent.appName - def listener = _listener.get - def start() { val gateway = parent.gatewayListener _listener = Some(new ExecutorsListener()) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index e9657cff34151..129a5f5bd3112 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -26,11 +26,11 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ private[ui] class IndexPage(parent: JobProgressUI) { + private lazy val appName = parent.appName + private lazy val isFairScheduler = parent.isFairScheduler + private lazy val listener = parent.listener private val live = parent.live private val sc = parent.sc - private def appName = parent.appName - private def isFairScheduler = parent.isFairScheduler - private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 1736b3a434124..64de63f7aae05 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,6 +29,9 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressUI(parent: SparkUI) { + lazy val appName = parent.appName + lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + lazy val listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val live = parent.live val sc = parent.sc @@ -38,10 +41,6 @@ private[ui] class JobProgressUI(parent: SparkUI) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None - def appName = parent.appName - def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - def listener = _listener.get - def start() { val gateway = parent.gatewayListener _listener = Some(new JobProgressListener(sc, live)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 431b4515c11f7..9ea72d6b473f4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,10 +26,10 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressUI) { + private lazy val appName = parent.appName + private lazy val listener = parent.listener private val live = parent.live private val sc = parent.sc - private def appName = parent.appName - private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index c88db232f6561..eb68b25662113 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,9 +28,9 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { + private lazy val appName = parent.appName + private lazy val listener = parent.listener private val dateFmt = parent.dateFmt - private def appName = parent.appName - private def listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index cee7426b9a9e5..4b59c9609045e 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -26,6 +26,8 @@ import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { + lazy val appName = parent.appName + lazy val listener = _listener.get val live = parent.live val sc = parent.sc @@ -33,9 +35,6 @@ private[ui] class BlockManagerUI(parent: SparkUI) { private val rddPage = new RDDPage(this) private var _listener: Option[BlockManagerListener] = None - def appName = parent.appName - def listener = _listener.get - def start() { val gateway = parent.gatewayListener _listener = Some(new BlockManagerListener) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index f24bdf366d94b..e47f9368da62f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,8 +28,8 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private def appName = parent.appName - private def listener = parent.listener + private lazy val appName = parent.appName + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { // Calculate macro-level statistics diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 290520a015fa0..210f4c0257347 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -28,8 +28,8 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private def appName = parent.appName - private def listener = parent.listener + private lazy val appName = parent.appName + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList From 4d2fb0c3b667284af31a27698083b2074d2797dc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 27 Feb 2014 17:49:38 -0800 Subject: [PATCH 36/68] Fix format fail --- .../main/scala/org/apache/spark/storage/StorageUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index d89d5061e4849..94ac8caeacb24 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -98,8 +98,8 @@ object StorageUtils { val rddInfos = blockStatusMap.map { case (rddId, blocks) => // Add up memory and disk sizes val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } - val memSize = persistedBlocks.map(_.memSize).reduceOption(_+_).getOrElse(0L) - val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) rddInfoMap.get(rddId).map { rddInfo => rddInfo.numCachedPartitions = persistedBlocks.length rddInfo.memSize = memSize From 0503e4b9e0b988d213e23d792e8f3a21415054d3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 28 Feb 2014 13:51:17 -0800 Subject: [PATCH 37/68] Fix PySpark tests + remove sc.clearFiles/clearJars The reason why PySpark tests failed was because this PR previously introduced a default parameter to a few methods in SparkContext, and this was not understood by the Py4j conversion of JavaSparkContext, since Java does not have default parameters. The fix gets rid of the use of default parameters, which also simplifies the logic of triggering SparkListenerEnvironmentUpdate events. This commit also deprecates sc.clearFiles and sc.clearJars, since they achieve little beyond deleting a few map entries when the SparkContext is already terminating anyway. --- .../scala/org/apache/spark/SparkContext.scala | 56 ++++++++----------- .../spark/api/java/JavaSparkContext.scala | 2 + 2 files changed, 25 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 91ced298a48d5..ecc0755c03601 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -157,7 +157,7 @@ class SparkContext( // Add each JAR given through the constructor if (jars != null) { - jars.foreach { jar => addJar(jar, updateEnvironment = false) } + jars.foreach(addJar) } private[spark] val executorMemory = conf.getOption("spark.executor.memory") @@ -627,7 +627,7 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, * use `SparkFiles.get(path)` to find its download location. */ - def addFile(path: String, updateEnvironment: Boolean = true) { + def addFile(path: String) { val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addFile(new File(uri.getPath)) @@ -640,9 +640,6 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) - if (updateEnvironment) { - updateEnvironmentProperties() - } } def addSparkListener(listener: SparkListener) { @@ -710,11 +707,9 @@ class SparkContext( * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ - def clearFiles(updateEnvironment: Boolean = true) { + @deprecated("added files are now temporary files and need not be deleted manually", "1.0.0") + def clearFiles() { addedFiles.clear() - if (updateEnvironment) { - updateEnvironmentProperties() - } } /** @@ -749,7 +744,7 @@ class SparkContext( * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported * filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node. */ - def addJar(path: String, updateEnvironment: Boolean = true) { + def addJar(path: String) { if (path == null) { logWarning("null specified as parameter to addJar") } else { @@ -763,8 +758,8 @@ class SparkContext( // A JAR file which exists only on the driver node case null | "file" => if (SparkHadoopUtil.get.isYarnMode() && master == "yarn-standalone") { - // In order for this to work in yarn standalone mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // In order for this to work in yarn standalone mode the user must specify the + // --addjars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { @@ -772,7 +767,7 @@ class SparkContext( } catch { case e: Exception => { // For now just log an error but allow to go through so spark examples work. - // The spark examples don't really need the jar distributed since its also + // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null @@ -793,20 +788,15 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } - if (updateEnvironment) { - updateEnvironmentProperties() - } } /** * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ - def clearJars(updateEnvironment: Boolean = true) { + @deprecated("added jars are now temporary files and need not be deleted manually", "1.0.0") + def clearJars() { addedJars.clear() - if (updateEnvironment) { - updateEnvironmentProperties() - } } /** Shut down the SparkContext. */ @@ -822,9 +812,6 @@ class SparkContext( taskScheduler = null // TODO: Cache.stop()? env.stop() - // Clean up locally linked files - clearFiles(updateEnvironment = false) - clearJars(updateEnvironment = false) SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() @@ -1047,17 +1034,20 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() - /** Update environment properties and post the corresponding event to the DAG scheduler */ + /** + * Update environment properties and post the corresponding event to the DAG scheduler, + * if it is ready. + */ private def updateEnvironmentProperties() { - val schedulingMode = getSchedulingMode.toString - val addedJarPaths = addedJars.keys.toSeq - val addedFilePaths = addedFiles.keys.toSeq - val environmentDetails = - SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) - val environmentUpdate = new SparkListenerEnvironmentUpdate(environmentDetails) - - // DAG scheduler may not be ready yet - Option(dagScheduler).foreach(_.post(environmentUpdate)) + Option(dagScheduler).foreach { scheduler => + val schedulingMode = getSchedulingMode.toString + val addedJarPaths = addedJars.keys.toSeq + val addedFilePaths = addedFiles.keys.toSeq + val environmentDetails = + SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) + val environmentUpdate = new SparkListenerEnvironmentUpdate(environmentDetails) + scheduler.post(environmentUpdate) + } } /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dc26b7f621fee..f2e5a854833d9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -415,6 +415,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ + @deprecated("added jars are now temporary files and need not be deleted manually", "1.0.0") def clearJars() { sc.clearJars() } @@ -423,6 +424,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ + @deprecated("added files are now temporary files and need not be deleted manually", "1.0.0") def clearFiles() { sc.clearFiles() } From 5d2cec1daaca58c8c4e09a454db9a92d8d4cc1da Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 28 Feb 2014 15:23:35 -0800 Subject: [PATCH 38/68] JobLogger: ID -> Id --- .../apache/spark/scheduler/JobLogger.scala | 94 +++++++++---------- .../spark/scheduler/JobLoggerSuite.scala | 44 ++++----- 2 files changed, 69 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index a7d2342e2d079..a78d3c4bda291 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -47,9 +47,9 @@ class JobLogger(val user: String, val logDirName: String) "/tmp/spark-%s".format(user) } - private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIDToJobID = new HashMap[Int, Int] - private val jobIDToStageIDs = new HashMap[Int, Seq[Int]] + private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIdToJobId = new HashMap[Int, Int] + private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] @@ -57,9 +57,9 @@ class JobLogger(val user: String, val logDirName: String) // The following 5 functions are used only in testing. private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIDToPrintWriter = jobIDToPrintWriter - private[scheduler] def getStageIDToJobID = stageIDToJobID - private[scheduler] def getJobIDToStageIDs = jobIDToStageIDs + private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter + private[scheduler] def getStageIdToJobId = stageIdToJobId + private[scheduler] def getJobIdToStageIds = jobIdToStageIds private[scheduler] def getEventQueue = eventQueue /** Create a folder for log files, the folder's name is the creation time of jobLogger */ @@ -76,78 +76,78 @@ class JobLogger(val user: String, val logDirName: String) /** * Create a log file for one job - * @param jobID ID of the job + * @param jobId ID of the job * @exception FileNotFoundException Fail to create log file */ - protected def createLogWriter(jobID: Int) { + protected def createLogWriter(jobId: Int) { try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) - jobIDToPrintWriter += (jobID -> fileWriter) + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) + jobIdToPrintWriter += (jobId -> fileWriter) } catch { case e: FileNotFoundException => e.printStackTrace() } } /** - * Close log file, and clean the stage relationship in stageIDToJobID - * @param jobID ID of the job + * Close log file, and clean the stage relationship in stageIdToJobId + * @param jobId ID of the job */ - protected def closeLogWriter(jobID: Int) { - jobIDToPrintWriter.get(jobID).foreach { fileWriter => + protected def closeLogWriter(jobId: Int) { + jobIdToPrintWriter.get(jobId).foreach { fileWriter => fileWriter.close() - jobIDToStageIDs.get(jobID).foreach(_.foreach { stageID => - stageIDToJobID -= stageID + jobIdToStageIds.get(jobId).foreach(_.foreach { stageId => + stageIdToJobId -= stageId }) - jobIDToPrintWriter -= jobID - jobIDToStageIDs -= jobID + jobIdToPrintWriter -= jobId + jobIdToStageIds -= jobId } } /** * Build up the maps that represent stage-job relationships - * @param jobID ID of the job - * @param stageIDs IDs of the associated stages + * @param jobId ID of the job + * @param stageIds IDs of the associated stages */ - protected def buildJobStageDependencies(jobID: Int, stageIDs: Seq[Int]) = { - jobIDToStageIDs(jobID) = stageIDs - stageIDs.foreach { stageID => stageIDToJobID(stageID) = jobID } + protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { + jobIdToStageIds(jobId) = stageIds + stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } } /** * Write info into log file - * @param jobID ID of the job + * @param jobId ID of the job * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + info } - jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) } /** * Write info into log file - * @param stageID ID of the stage + * @param stageId ID of the stage * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) { - stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) + protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { + stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) } /** * Record task metrics into job log files, including execution info and shuffle metrics - * @param stageID Stage ID of the task + * @param stageId Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info * @param taskMetrics Task running metrics */ - protected def recordTaskMetrics(stageID: Int, status: String, + protected def recordTaskMetrics(stageId: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime @@ -166,7 +166,7 @@ class JobLogger(val user: String, val logDirName: String) case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) } /** @@ -184,8 +184,8 @@ class JobLogger(val user: String, val logDirName: String) * @param stageCompleted Stage completed event */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - val stageID = stageCompleted.stageInfo.stageId - stageLogInfo(stageID, "STAGE_ID=%d STATUS=COMPLETED".format(stageID)) + val stageId = stageCompleted.stageInfo.stageId + stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) } /** @@ -217,8 +217,8 @@ class JobLogger(val user: String, val logDirName: String) * @param jobEnd Job end event */ override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val jobID = jobEnd.jobId - var info = "JOB_ID=" + jobID + val jobId = jobEnd.jobId + var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -226,19 +226,19 @@ class JobLogger(val user: String, val logDirName: String) exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(jobID, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(jobID) + jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(jobId) } /** * Record job properties into job log file - * @param jobID ID of the job + * @param jobId ID of the job * @param properties Properties of the job */ - protected def recordJobProperties(jobID: Int, properties: Properties) { + protected def recordJobProperties(jobId: Int, properties: Properties) { if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobID, description, false) + jobLogInfo(jobId, description, false) } } @@ -247,11 +247,11 @@ class JobLogger(val user: String, val logDirName: String) * @param jobStart Job start event */ override def onJobStart(jobStart: SparkListenerJobStart) { - val jobID = jobStart.jobId + val jobId = jobStart.jobId val properties = jobStart.properties - createLogWriter(jobID) - recordJobProperties(jobID, properties) - buildJobStageDependencies(jobID, jobStart.stageIds) - jobLogInfo(jobID, "JOB_ID=" + jobID + " STATUS=STARTED") + createLogWriter(jobId) + recordJobProperties(jobId, properties) + buildJobStageDependencies(jobId, jobStart.stageIds) + jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index d0c22cc937a16..1ee5b67e2ff31 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -30,8 +30,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers test("inner method") { sc = new SparkContext("local", "joblogger") val joblogger = new JobLogger { - def createLogWriterTest(jobID: Int) = createLogWriter(jobID) - def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) + def createLogWriterTest(jobId: Int) = createLogWriter(jobId) + def closeLogWriterTest(jobId: Int) = closeLogWriter(jobId) } type MyRDD = RDD[(Int, Int)] def makeRdd(numPartitions: Int, dependencies: List[Dependency[_]]): MyRDD = { @@ -44,34 +44,34 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers }).toArray } } - val jobID = 5 + val jobId = 5 val parentRdd = makeRdd(4, Nil) val shuffleDep = new ShuffleDependency(parentRdd, null) val rootRdd = makeRdd(4, List(shuffleDep)) val shuffleMapStage = - new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobID, None) + new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobId, None) val rootStage = - new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None) + new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobId, None) val rootStageInfo = StageInfo.fromStage(rootStage) - joblogger.onJobStart(SparkListenerJobStart(jobID, Seq[Int](0, 1))) + joblogger.onJobStart(SparkListenerJobStart(jobId, Seq[Int](0, 1))) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo)) - joblogger.createLogWriterTest(jobID) - joblogger.getJobIDToPrintWriter.size should be (1) - joblogger.getJobIDToStageIDs.get(jobID).get.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) - joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) - joblogger.closeLogWriterTest(jobID) - joblogger.getStageIDToJobID.size should be (0) - joblogger.getJobIDToStageIDs.size should be (0) - joblogger.getJobIDToPrintWriter.size should be (0) + joblogger.createLogWriterTest(jobId) + joblogger.getJobIdToPrintWriter.size should be (1) + joblogger.getJobIdToStageIds.get(jobId).get.size should be (2) + joblogger.getStageIdToJobId.get(0) should be (Some(jobId)) + joblogger.getStageIdToJobId.get(1) should be (Some(jobId)) + joblogger.closeLogWriterTest(jobId) + joblogger.getStageIdToJobId.size should be (0) + joblogger.getJobIdToStageIds.size should be (0) + joblogger.getJobIdToPrintWriter.size should be (0) } test("inner variables") { sc = new SparkContext("local[4]", "joblogger") val joblogger = new JobLogger { - override protected def closeLogWriter(jobID: Int) = - getJobIDToPrintWriter.get(jobID).foreach { fileWriter => + override protected def closeLogWriter(jobId: Int) = + getJobIdToPrintWriter.get(jobId).foreach { fileWriter => fileWriter.close() } } @@ -84,11 +84,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIDToPrintWriter.size should be (1) - joblogger.getStageIDToJobID.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(0)) - joblogger.getStageIDToJobID.get(1) should be (Some(0)) - joblogger.getJobIDToStageIDs.size should be (1) + joblogger.getJobIdToPrintWriter.size should be (1) + joblogger.getStageIdToJobId.size should be (2) + joblogger.getStageIdToJobId.get(0) should be (Some(0)) + joblogger.getStageIdToJobId.get(1) should be (Some(0)) + joblogger.getJobIdToStageIds.size should be (1) } From 2981d611e6802d8b5a1e3ae0de2fa97de1342956 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 28 Feb 2014 20:17:11 -0800 Subject: [PATCH 39/68] Move SparkListenerBus out of DAGScheduler + Clean up This PR introduces new SparkListenerEvents that are generated outside of DAGSchduler. Instead of going through multiple layers (SparkContext -> DAGScheduler -> SparkListenerBus) to post the event, we post them directly to sc.listenerBus (SparkContext -> SparkListenerBus). This commit also cleans up the initialization order of the UI and the schedulers in SparkContext, as well as variable names in DAGScheduler. Further, some tests create events with null TaskInfo, which causes NPE on certain UI listeners. This is now fixed. --- .../scala/org/apache/spark/SparkContext.scala | 50 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 279 +++++++++--------- .../spark/scheduler/DAGSchedulerSource.scala | 6 +- .../BlockManagerRegistrationListener.scala | 2 +- .../org/apache/spark/ui/UISparkListener.scala | 15 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 42 +-- .../spark/ui/jobs/JobProgressListener.scala | 149 +++++----- .../org/apache/spark/util/JsonProtocol.scala | 40 +-- .../apache/spark/JobCancellationSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 11 +- .../spark/scheduler/JobLoggerSuite.scala | 4 +- .../spark/scheduler/SparkListenerSuite.scala | 12 +- .../spark/scheduler/TaskSetManagerSuite.scala | 6 +- .../ui/jobs/JobProgressListenerSuite.scala | 12 +- 14 files changed, 327 insertions(+), 307 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ecc0755c03601..c58852294668e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -195,19 +195,22 @@ class SparkContext( } executorEnvs("SPARK_USER") = sparkUser + // An asynchronous listener bus for Spark events + private[spark] val listenerBus = new SparkListenerBus + + // Start the UI before posting events to listener bus, because the UI listens for Spark events + ui.start() + // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName) taskScheduler.start() - @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) - - // Start the UI before the DAG scheduler, because the UI listens for Spark events - ui.start() - + @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() - dagScheduler.post(new SparkListenerApplicationStart(appName)) - updateEnvironmentProperties() + // Post initialization events + postApplicationStartEvent() + postEnvironmentUpdateEvent() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -640,10 +643,11 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) + postEnvironmentUpdateEvent() } def addSparkListener(listener: SparkListener) { - dagScheduler.addSparkListener(listener) + listenerBus.addListener(listener) } /** @@ -670,7 +674,7 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage,StageInfo] = { + def getStageInfo: Map[Stage, StageInfo] = { dagScheduler.stageToInfos } @@ -707,7 +711,7 @@ class SparkContext( * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ - @deprecated("added files are now temporary files and need not be deleted manually", "1.0.0") + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { addedFiles.clear() } @@ -736,7 +740,7 @@ class SparkContext( val rddId = rdd.id env.blockManager.master.removeRdd(rddId, blocking) persistentRdds.remove(rddId) - dagScheduler.post(new SparkListenerUnpersistRDD(rddId)) + listenerBus.post(SparkListenerUnpersistRDD(rddId)) } /** @@ -788,13 +792,14 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } + postEnvironmentUpdateEvent() } /** * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ - @deprecated("added jars are now temporary files and need not be deleted manually", "1.0.0") + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { addedJars.clear() } @@ -1034,19 +1039,24 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() - /** - * Update environment properties and post the corresponding event to the DAG scheduler, - * if it is ready. - */ - private def updateEnvironmentProperties() { - Option(dagScheduler).foreach { scheduler => + /** Post the application start event if the listener bus is ready */ + private def postApplicationStartEvent() { + Option(listenerBus).foreach { bus => + val applicationStart = SparkListenerApplicationStart(appName) + bus.post(applicationStart) + } + } + + /** Post the environment update event if the listener bus is ready */ + private def postEnvironmentUpdateEvent() { + Option(listenerBus).foreach { bus => val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq val addedFilePaths = addedFiles.keys.toSeq val environmentDetails = SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) - val environmentUpdate = new SparkListenerEnvironmentUpdate(environmentDetails) - scheduler.post(environmentUpdate) + val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) + bus.post(environmentUpdate) } } 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 81c88b0f9b40c..a3ab97d2b5aff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -54,87 +54,51 @@ import org.apache.spark.util.{Utils, MetadataCleaner, MetadataCleanerType, TimeS */ private[spark] class DAGScheduler( - taskSched: TaskScheduler, + taskScheduler: TaskScheduler, + listenerBus: SparkListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv) extends Logging { - def this(taskSched: TaskScheduler) { - this(taskSched, SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], - SparkEnv.get.blockManager.master, SparkEnv.get) - } - taskSched.setDAGScheduler(this) - - // Called by TaskScheduler to report task's starting. - def taskStarted(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! BeginEvent(task, taskInfo) - } - - // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) - } - - // Called by TaskScheduler to report task completions or failures. - def taskEnded( - task: Task[_], - reason: TaskEndReason, - result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { - eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) - } - - // Called by TaskScheduler when an executor fails. - def executorLost(execId: String) { - eventProcessActor ! ExecutorLost(execId) - } + import DAGScheduler._ - // Called by TaskScheduler when a host is added - def executorGained(execId: String, host: String) { - eventProcessActor ! ExecutorGained(execId, host) - } - - // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or - // cancellation of the job itself. - def taskSetFailed(taskSet: TaskSet, reason: String) { - eventProcessActor ! TaskSetFailed(taskSet, reason) + def this(sc: SparkContext) = { + this( + sc.taskScheduler, + sc.listenerBus, + sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], + sc.env.blockManager.master, + sc.env) } - // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; - // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one - // as more failure events come in - val RESUBMIT_TIMEOUT = 200.milliseconds - - // The time, in millis, to wake up between polls of the completion queue in order to potentially - // resubmit failed stages - val POLL_TIMEOUT = 10L - - // Warns the user if a stage contains a task with size greater than this value (in KB) - val TASK_SIZE_TO_WARN = 100 - private var eventProcessActor: ActorRef = _ private[scheduler] val nextJobId = new AtomicInteger(0) - - def numTotalJobs: Int = nextJobId.get() - + private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - + private[scheduler] val stageIdToActiveJob = new HashMap[Int, ActiveJob] + private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - // An async scheduler event bus. The bus should be stopped when DAGSCheduler is stopped. - private[spark] val listenerBus = new SparkListenerBus + // Stages we need to run whose parents aren't done + private[scheduler] val waitingStages = new HashSet[Stage] + + // Stages we are running right now + private[scheduler] val runningStages = new HashSet[Stage] + + // Stages that must be resubmitted due to fetch failures + private[scheduler] val failedStages = new HashSet[Stage] + + // Missing tasks from each stage + private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + + private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] @@ -145,22 +109,12 @@ class DAGScheduler( // // TODO: Garbage collect information about failure epochs when we know there are no more // stray messages to detect. - val failedEpoch = new HashMap[String, Long] - - // stage id to the active job - val idToActiveJob = new HashMap[Int, ActiveJob] - - val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done - val running = new HashSet[Stage] // Stages we are running right now - val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - // Missing tasks from each stage - val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + private val failedEpoch = new HashMap[String, Long] - val activeJobs = new HashSet[ActiveJob] - val resultStageToJob = new HashMap[Stage, ActiveJob] + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) + taskScheduler.setDAGScheduler(this) /** * Starts the event processing actor. The actor has two responsibilities: @@ -199,12 +153,41 @@ class DAGScheduler( blockManagerMaster.registrationListener.foreach(_.setListenerBus(listenerBus)) } - def addSparkListener(listener: SparkListener) { - listenerBus.addListener(listener) + // Called by TaskScheduler to report task's starting. + def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! BeginEvent(task, taskInfo) + } + + // Called to report that a task has completed and results are being fetched remotely. + def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(task, taskInfo) + } + + // Called by TaskScheduler to report task completions or failures. + def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) + } + + // Called by TaskScheduler when an executor fails. + def executorLost(execId: String) { + eventProcessActor ! ExecutorLost(execId) + } + + // Called by TaskScheduler when a host is added + def executorGained(execId: String, host: String) { + eventProcessActor ! ExecutorGained(execId, host) } - def post(event: SparkListenerEvent) { - listenerBus.post(event) + // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or + // cancellation of the job itself. + def taskSetFailed(taskSet: TaskSet, reason: String) { + eventProcessActor ! TaskSetFailed(taskSet, reason) } private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { @@ -383,9 +366,9 @@ class DAGScheduler( def removeStage(stageId: Int) { // data structures based on Stage for (stage <- stageIdToStage.get(stageId)) { - if (running.contains(stage)) { + if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) - running -= stage + runningStages -= stage } stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { @@ -395,13 +378,13 @@ class DAGScheduler( logDebug("Removing pending status for stage %d".format(stageId)) } pendingTasks -= stage - if (waiting.contains(stage)) { + if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= stage + waitingStages -= stage } - if (failed.contains(stage)) { + if (failedStages.contains(stage)) { logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= stage + failedStages -= stage } } // data structures based on StageId @@ -551,13 +534,13 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { - idToActiveJob(jobId) = job + stageIdToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -574,9 +557,9 @@ class DAGScheduler( case AllJobsCancelled => // Cancel all running jobs. - running.map(_.jobId).foreach { handleJobCancellation } + runningStages.map(_.jobId).foreach { handleJobCancellation } activeJobs.clear() // These should already be empty by this point, - idToActiveJob.clear() // but just in case we lost track of some jobs... + stageIdToActiveJob.clear() // but just in case we lost track of some jobs... case ExecutorGained(execId, host) => handleExecutorGained(execId, host) @@ -586,7 +569,7 @@ class DAGScheduler( case BeginEvent(task, taskInfo) => for ( - job <- idToActiveJob.get(task.stageId); + job <- stageIdToActiveJob.get(task.stageId); stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage) ) { @@ -598,22 +581,22 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) } } - post(SparkListenerTaskStart(task.stageId, taskInfo)) + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) case GettingResultEvent(task, taskInfo) => - post(SparkListenerTaskGettingResult(taskInfo)) + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } case ResubmitFailedStages => - if (failed.size > 0) { + if (failedStages.size > 0) { // Failed stages may be removed by job cancellation, so failed might be empty even if // the ResubmitFailedStages event has been scheduled. resubmitFailedStages() @@ -624,7 +607,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) } return true } @@ -638,9 +621,9 @@ class DAGScheduler( private[scheduler] def resubmitFailedStages() { logInfo("Resubmitting failed stages") clearCacheLocs() - val failed2 = failed.toArray - failed.clear() - for (stage <- failed2.sortBy(_.jobId)) { + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -653,12 +636,12 @@ class DAGScheduler( // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") - logTrace("running: " + running) - logTrace("waiting: " + waiting) - logTrace("failed: " + failed) - val waiting2 = waiting.toArray - waiting.clear() - for (stage <- waiting2.sortBy(_.jobId)) { + logTrace("running: " + runningStages) + logTrace("waiting: " + waitingStages) + logTrace("failed: " + failedStages) + val waitingStagesCopy = waitingStages.toArray + waitingStages.clear() + for (stage <- waitingStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -702,7 +685,7 @@ class DAGScheduler( stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through stageToInfos -= s // completion events or stage abort jobIdToStageIds -= job.jobId - post(SparkListenerJobEnd(job.jobId, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } } @@ -714,7 +697,7 @@ class DAGScheduler( private def activeJobForStage(stage: Stage): Option[Int] = { if (stageIdToJobIds.contains(stage.id)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(idToActiveJob.contains(_)) + jobsThatUseStage.find(stageIdToActiveJob.contains) } else { None } @@ -725,18 +708,18 @@ class DAGScheduler( val jobId = activeJobForStage(stage) if (jobId.isDefined) { logDebug("submitStage(" + stage + ")") - if (!waiting(stage) && !running(stage) && !failed(stage)) { + if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - running += stage + runningStages += stage } else { for (parent <- missing) { submitStage(parent) } - waiting += stage + waitingStages += stage } } } else { @@ -767,8 +750,8 @@ class DAGScheduler( } } - val properties = if (idToActiveJob.contains(jobId)) { - idToActiveJob(stage.jobId).properties + val properties = if (stageIdToActiveJob.contains(jobId)) { + stageIdToActiveJob(stage.jobId).properties } else { //this stage will be assigned to "default" pool null @@ -776,7 +759,7 @@ class DAGScheduler( // must be run listener before possible NotSerializableException // should be "StageSubmitted" first and then "JobEnded" - post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) + listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) if (tasks.size > 0) { // Preemptively serialize a task to make sure it can be serialized. We are catching this @@ -788,20 +771,20 @@ class DAGScheduler( } catch { case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) - running -= stage + runningStages -= stage return } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - taskSched.submitTasks( + taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) - running -= stage + runningStages -= stage } } @@ -825,8 +808,8 @@ class DAGScheduler( } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) - post(SparkListenerStageCompleted(stageToInfos(stage))) - running -= stage + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + runningStages -= stage } event.reason match { case Success => @@ -845,12 +828,12 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - idToActiveJob -= stage.jobId + stageIdToActiveJob -= stage.jobId activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) jobIdToStageIdsRemove(job.jobId) - post(SparkListenerJobEnd(job.jobId, JobSucceeded)) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -867,12 +850,12 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (running.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") - logInfo("running: " + running) - logInfo("waiting: " + waiting) - logInfo("failed: " + failed) + logInfo("running: " + runningStages) + logInfo("waiting: " + waitingStages) + logInfo("failed: " + failedStages) if (stage.shuffleDep.isDefined) { // We supply true to increment the epoch number here in case this is a // recomputation of the map outputs. In that case, some nodes may have cached @@ -895,14 +878,14 @@ class DAGScheduler( submitStage(stage) } else { val newlyRunnable = new ArrayBuffer[Stage] - for (stage <- waiting) { + for (stage <- waitingStages) { logInfo("Missing parents for " + stage + ": " + getMissingParentStages(stage)) } - for (stage <- waiting if getMissingParentStages(stage) == Nil) { + for (stage <- waitingStages if getMissingParentStages(stage) == Nil) { newlyRunnable += stage } - waiting --= newlyRunnable - running ++= newlyRunnable + waitingStages --= newlyRunnable + runningStages ++= newlyRunnable for { stage <- newlyRunnable.sortBy(_.id) jobId <- activeJobForStage(stage) @@ -921,7 +904,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - running -= failedStage + runningStages -= failedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") @@ -933,7 +916,7 @@ class DAGScheduler( } logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") - if (failed.isEmpty && eventProcessActor != null) { + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. @@ -941,8 +924,8 @@ class DAGScheduler( env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } - failed += failedStage - failed += mapStage + failedStages += failedStage + failedStages += mapStage // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) @@ -990,7 +973,7 @@ class DAGScheduler( // Block manager master actor should not be null except during tests if (blockManagerMaster.driverActor != null) { val storageStatusList = blockManagerMaster.getStorageStatus - post(new SparkListenerExecutorsStateChange(storageStatusList)) + listenerBus.post(SparkListenerExecutorsStateChange(storageStatusList)) } } @@ -1009,14 +992,14 @@ class DAGScheduler( logDebug("Trying to cancel unregistered job " + jobId) } else { val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach { taskSched.cancelTasks } + independentStages.foreach(taskScheduler.cancelTasks) val error = new SparkException("Job %d cancelled".format(jobId)) - val job = idToActiveJob(jobId) + val job = stageIdToActiveJob(jobId) job.listener.jobFailed(error) jobIdToStageIds -= jobId activeJobs -= job - idToActiveJob -= jobId - post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + stageIdToActiveJob -= jobId + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1036,10 +1019,10 @@ class DAGScheduler( val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) jobIdToStageIdsRemove(job.jobId) - idToActiveJob -= resultStage.jobId + stageIdToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage - post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1130,7 +1113,21 @@ class DAGScheduler( eventProcessActor ! StopDAGScheduler } metadataCleaner.cancel() - taskSched.stop() + taskScheduler.stop() listenerBus.stop() } } + +private[spark] object DAGScheduler { + // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; + // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one + // as more failure events come in + val RESUBMIT_TIMEOUT = 200.milliseconds + + // The time, in millis, to wake up between polls of the completion queue in order to potentially + // resubmit failed stages + val POLL_TIMEOUT = 10L + + // Warns the user if a stage contains a task with size greater than this value (in KB) + val TASK_SIZE_TO_WARN = 100 +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index b52fe2410abde..5878e733908f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -28,15 +28,15 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: Spar val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + override def getValue: Int = dagScheduler.failedStages.size }) metricRegistry.register(MetricRegistry.name("stage", "runningStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + override def getValue: Int = dagScheduler.runningStages.size }) metricRegistry.register(MetricRegistry.name("stage", "waitingStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + override def getValue: Int = dagScheduler.waitingStages.size }) metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala index 31ad13cacdb79..4643ae29baa17 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala @@ -43,7 +43,7 @@ private[spark] class BlockManagerRegistrationListener { * post the event; otherwise, buffer it. */ def onBlockManagerRegister(storageStatus: Array[StorageStatus]) { - val executorsStateChange = new SparkListenerExecutorsStateChange(storageStatus) + val executorsStateChange = SparkListenerExecutorsStateChange(storageStatus) _listenerBus.map(_.post(executorsStateChange)).getOrElse { bufferedEvents += executorsStateChange } diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 02f42f0b3592e..4549ffce85a40 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -151,12 +151,15 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val execId = taskEnd.taskInfo.executorId - val metrics = taskEnd.taskMetrics - if (metrics != null) { - val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq()) - if (updatedBlocks.length > 0) { - updateStorageStatus(execId, updatedBlocks) + val info = taskEnd.taskInfo + if (info != null) { + val execId = info.executorId + val metrics = taskEnd.taskMetrics + if (metrics != null) { + val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq()) + if (updatedBlocks.length > 0) { + updateStorageStatus(execId, updatedBlocks) + } } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 0da815c402f87..a838dee66e69a 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -176,28 +176,32 @@ private[ui] class ExecutorsListener extends StorageStatusSparkListener { } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = formatExecutorId(taskEnd.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - if (taskEnd.taskMetrics != null) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 } - taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } } + super.onTaskEnd(taskEnd) } - super.onTaskEnd(taskEnd) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 54193b5057ded..e191883e433b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -123,11 +123,13 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val sid = taskStart.stageId val taskInfo = taskStart.taskInfo - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - tasksActive(taskInfo.taskId) = taskInfo - val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) - taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) - stageIdToTaskInfos(sid) = taskMap + if (taskInfo != null) { + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + tasksActive(taskInfo.taskId) = taskInfo + val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) + stageIdToTaskInfos(sid) = taskMap + } } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) @@ -138,81 +140,82 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val sid = taskEnd.stageId + val info = taskEnd.taskInfo + + if (info != null) { + // create executor summary map if necessary + val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, + op = new HashMap[String, ExecutorSummary]()) + executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary()) + + val executorSummary = executorSummaryMap.get(info.executorId) + executorSummary match { + case Some(y) => { + // first update failed-task, succeed-task + taskEnd.reason match { + case Success => + y.succeededTasks += 1 + case _ => + y.failedTasks += 1 + } + + // update duration + y.taskTime += info.duration + + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } + metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } + y.memoryBytesSpilled += metrics.memoryBytesSpilled + y.diskBytesSpilled += metrics.diskBytesSpilled + } + } + case _ => {} + } - // create executor summary map if necessary - val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, - op = new HashMap[String, ExecutorSummary]()) - executorSummaryMap.getOrElseUpdate(key = taskEnd.taskInfo.executorId, - op = new ExecutorSummary()) + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage + tasksActive.remove(info.taskId) - val executorSummary = executorSummaryMap.get(taskEnd.taskInfo.executorId) - executorSummary match { - case Some(y) => { - // first update failed-task, succeed-task + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { - case Success => - y.succeededTasks += 1 + case e: ExceptionFailure => + stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) case _ => - y.failedTasks += 1 + stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 + (None, Option(taskEnd.taskMetrics)) } - // update duration - y.taskTime += taskEnd.taskInfo.duration - - val taskMetrics = taskEnd.taskMetrics - if (taskMetrics != null) { - taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } - taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } - y.memoryBytesSpilled += taskMetrics.memoryBytesSpilled - y.diskBytesSpilled += taskMetrics.diskBytesSpilled - } - } - case _ => {} + stageIdToTime.getOrElseUpdate(sid, 0L) + val time = metrics.map(m => m.executorRunTime).getOrElse(0L) + stageIdToTime(sid) += time + totalTime += time + + stageIdToShuffleRead.getOrElseUpdate(sid, 0L) + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) + stageIdToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead + + stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) + stageIdToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite + + stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) + val memoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled).getOrElse(0L) + stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled + + stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) + val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) + stageIdToDiskBytesSpilled(sid) += diskBytesSpilled + + val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(info.taskId) = new TaskUIData(info, metrics, failureInfo) + stageIdToTaskInfos(sid) = taskMap } - - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage - tasksActive.remove(taskEnd.taskInfo.taskId) - - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } - - stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(m => m.executorRunTime).getOrElse(0L) - stageIdToTime(sid) += time - totalTime += time - - stageIdToShuffleRead.getOrElseUpdate(sid, 0L) - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) - stageIdToShuffleRead(sid) += shuffleRead - totalShuffleRead += shuffleRead - - stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) - stageIdToShuffleWrite(sid) += shuffleWrite - totalShuffleWrite += shuffleWrite - - stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) - val memoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled).getOrElse(0L) - stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled - - stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) - val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) - stageIdToDiskBytesSpilled(sid) += diskBytesSpilled - - val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) - val taskInfo = taskEnd.taskInfo - taskMap(taskInfo.taskId) = new TaskUIData(taskInfo, metrics, failureInfo) - stageIdToTaskInfos(sid) = taskMap } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { 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 e10b41a62e121..07fb80e5e6a9a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -83,28 +83,31 @@ private[spark] object JsonProtocol { } def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { - val taskInfo = taskInfoToJson(taskStart.taskInfo) + val taskInfo = taskStart.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfo) + ("Task Info" -> taskInfoJson) } def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { - val taskInfo = taskInfoToJson(taskGettingResult.taskInfo) + val taskInfo = taskGettingResult.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfo) + ("Task Info" -> taskInfoJson) } def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) - val taskInfo = taskInfoToJson(taskEnd.taskInfo) + val taskInfo = taskEnd.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfo) ~ + ("Task Info" -> taskInfoJson) ~ ("Task Metrics" -> taskMetricsJson) } @@ -166,7 +169,8 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoToJson(stageInfo.rddInfo) val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => val metricsJson = if (metrics != null) taskMetricsToJson(metrics) else JNothing - ("Task Info" -> taskInfoToJson(info)) ~ + val infoJson = if (info != null) taskInfoToJson(info) else JNothing + ("Task Info" -> infoJson) ~ ("Task Metrics" -> metricsJson) }.toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) @@ -410,23 +414,23 @@ private[spark] object JsonProtocol { def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { val stageInfo = stageInfoFromJson(json \ "Stage Info") val properties = propertiesFromJson(json \ "Properties") - new SparkListenerStageSubmitted(stageInfo, properties) + SparkListenerStageSubmitted(stageInfo, properties) } def stageCompletedFromJson(json: JValue): SparkListenerStageCompleted = { val stageInfo = stageInfoFromJson(json \ "Stage Info") - new SparkListenerStageCompleted(stageInfo) + SparkListenerStageCompleted(stageInfo) } def taskStartFromJson(json: JValue): SparkListenerTaskStart = { val stageId = (json \ "Stage ID").extract[Int] val taskInfo = taskInfoFromJson(json \ "Task Info") - new SparkListenerTaskStart(stageId, taskInfo) + SparkListenerTaskStart(stageId, taskInfo) } def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { val taskInfo = taskInfoFromJson(json \ "Task Info") - new SparkListenerTaskGettingResult(taskInfo) + SparkListenerTaskGettingResult(taskInfo) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { @@ -435,24 +439,24 @@ private[spark] object JsonProtocol { val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") val taskInfo = taskInfoFromJson(json \ "Task Info") val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") - new SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) } def jobStartFromJson(json: JValue): SparkListenerJobStart = { val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - new SparkListenerJobStart(jobId, stageIds, properties) + SparkListenerJobStart(jobId, stageIds, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { val jobId = (json \ "Job ID").extract[Int] val jobResult = jobResultFromJson(json \ "Job Result") - new SparkListenerJobEnd(jobId, jobResult) + SparkListenerJobEnd(jobId, jobResult) } def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { - new SparkListenerApplicationStart((json \ "App Name").extract[String]) + SparkListenerApplicationStart((json \ "App Name").extract[String]) } def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { @@ -461,17 +465,17 @@ private[spark] object JsonProtocol { "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, "System Properties" -> mapFromJson(json \ "System Properties").toSeq, "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) - new SparkListenerEnvironmentUpdate(environmentDetails) + SparkListenerEnvironmentUpdate(environmentDetails) } def executorsStateChangeFromJson(json: JValue): SparkListenerExecutorsStateChange = { val storageStatusList = (json \ "Storage Status List").extract[List[JValue]].map(storageStatusFromJson) - new SparkListenerExecutorsStateChange(storageStatusList) + SparkListenerExecutorsStateChange(storageStatusList) } def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { - new SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) + SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } /** diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 20c503d30c801..713ef69f35edb 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -89,7 +89,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -161,7 +161,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -191,7 +191,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } 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 ad890b4e4d08e..e5dfa6acfa40f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -94,7 +94,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { + scheduler = new DAGScheduler(taskScheduler, sc.listenerBus, mapOutputTracker, + blockManagerMaster, sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing runLocallyWithinThread(job) @@ -422,15 +423,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def assertDataStructuresEmpty = { assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) - assert(scheduler.failed.isEmpty) - assert(scheduler.idToActiveJob.isEmpty) + assert(scheduler.failedStages.isEmpty) + assert(scheduler.stageIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) assert(scheduler.stageToInfos.isEmpty) assert(scheduler.resultStageToJob.isEmpty) - assert(scheduler.running.isEmpty) + assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) - assert(scheduler.waiting.isEmpty) + assert(scheduler.waitingStages.isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 1ee5b67e2ff31..012f6ef4de3f0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -79,7 +79,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } rdd.reduceByKey(_+_).collect() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) @@ -110,7 +110,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } rdd.reduceByKey(_+_).collect() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) joblogger.onJobStartCount should be (1) joblogger.onJobEndCount should be (1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index a4c38d94e041c..bbd455155c975 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -46,7 +46,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc rdd2.setName("Target RDD") rdd2.count - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val first = listener.stageInfos.head @@ -65,7 +65,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val rdd2 = rdd1.map(x => x.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1), true) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val first = listener.stageInfos.head @@ -86,7 +86,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val d = sc.parallelize(0 to 1e4.toInt, 64).map{i => w(i)} d.count() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (1) val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1") @@ -98,7 +98,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc d4.collectAsMap() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (4) listener.stageInfos.foreach { stageInfo => /* small test, so some tasks might take less than 1 millisecond, but average should be greater @@ -146,7 +146,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x) assert(result === 1.to(akkaFrameSize).toArray) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.contains(TASK_INDEX)) @@ -161,7 +161,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.isEmpty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 34a7d8cefeea2..98b7ac758cb29 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,7 +26,9 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.FakeClock -class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) { +class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeClusterScheduler) + extends DAGScheduler(sc) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { taskScheduler.startedTasks += taskInfo.index } @@ -66,7 +68,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* val executors = new mutable.HashMap[String, String] ++ liveExecutors - dagScheduler = new FakeDAGScheduler(this) + dagScheduler = new FakeDAGScheduler(sc, this) def removeExecutor(execId: String): Unit = executors -= execId 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 006353b705e32..e632630ff3202 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 @@ -41,8 +41,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskInfo.finishTime = 1 var task = new ShuffleMapTask(0, null, null, 0, null) val taskType = Utils.getFormattedClassName(task) - listener.onTaskEnd( - new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 1000) @@ -50,8 +49,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) - listener.onTaskEnd( - new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.size == 1) // finish this task, should get updated duration @@ -60,8 +58,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) - listener.onTaskEnd( - new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 2000) @@ -71,8 +68,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) - listener.onTaskEnd( - new SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) .shuffleRead == 1000) } From 2fee310538da401156c2823af61ca0e740e1b78e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 28 Feb 2014 21:18:48 -0800 Subject: [PATCH 40/68] Address Patrick's comments This mainly involves (1) making event logging configurable and (2) setting the log boundary to be on the granularity of application rather than job. Other more minor changes include variable name changes, and directly assigning TaskMetrics.updatedBlocks rather than appending to it. --- .../scala/org/apache/spark/Aggregator.scala | 2 -- .../scala/org/apache/spark/CacheManager.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 12 ++++---- .../scala/org/apache/spark/ui/SparkUI.scala | 3 +- .../org/apache/spark/ui/UISparkListener.scala | 30 +++++++++---------- .../org/apache/spark/util/FileLogger.scala | 3 +- docs/configuration.md | 16 +++++++++- 7 files changed, 40 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c4579cf6ad560..ceead59b79ed6 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.{Option, deprecated} - import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 0876830fe0586..579418bb19a0d 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -80,7 +80,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // Update task metrics to include any blocks whose storage status is updated val metrics = context.taskMetrics if (metrics != null) { - metrics.updatedBlocks = Some(updatedBlocks ++ metrics.updatedBlocks.getOrElse(Seq())) + metrics.updatedBlocks = Some(updatedBlocks) } elements.iterator.asInstanceOf[Iterator[T]] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index dd77b5394e16c..b8a5c1daaa3ef 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -151,7 +151,7 @@ private[spark] class BlockManager( private def reportAllBlocks() { logInfo("Reporting " + blockInfo.size + " blocks to the master.") for ((blockId, info) <- blockInfo) { - val status = getUpdatedBlockStatus(blockId, info) + val status = getCurrentBlockStatus(blockId, info) if (!tryToReportBlockStatus(blockId, info, status)) { logError("Failed to report " + blockId + " to master; giving up.") return @@ -251,7 +251,7 @@ private[spark] class BlockManager( * the block is dropped from memory and possibly added to disk, return the new storage level * and the updated in-memory and on-disk sizes. */ - private def getUpdatedBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { + private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { val (newLevel, inMemSize, onDiskSize) = info.synchronized { info.level match { case null => @@ -626,7 +626,7 @@ private[spark] class BlockManager( // and tell the master about it. marked = true putBlockInfo.markReady(size) - val putBlockStatus = getUpdatedBlockStatus(blockId, putBlockInfo) + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } @@ -767,7 +767,7 @@ private[spark] class BlockManager( logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") } - val status = getUpdatedBlockStatus(blockId, info) + val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster) { reportBlockStatus(blockId, info, status, droppedMemorySize) } @@ -812,7 +812,7 @@ private[spark] class BlockManager( } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { - val status = getUpdatedBlockStatus(blockId, info) + val status = getCurrentBlockStatus(blockId, info) reportBlockStatus(blockId, info, status) } } else { @@ -848,7 +848,7 @@ private[spark] class BlockManager( iterator.remove() logInfo("Dropped block " + id) } - val status = getUpdatedBlockStatus(id, info) + val status = getCurrentBlockStatus(id, info) reportBlockStatus(id, info, status) } } 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 594dce8eac9f3..c9d3251cc30da 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -74,7 +74,7 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { private var _gatewayListener: Option[GatewayUISparkListener] = None def gatewayListener = _gatewayListener.getOrElse { - val gateway = new GatewayUISparkListener(this, live) + val gateway = new GatewayUISparkListener(this, sc) _gatewayListener = Some(gateway) gateway } @@ -125,6 +125,7 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { def stop() { server.foreach(_.stop()) + _gatewayListener.foreach(_.stop()) logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 4549ffce85a40..4c10b0e98bb58 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -26,6 +26,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark.util.FileLogger import org.apache.spark.util.JsonProtocol +import org.apache.spark.SparkContext private[ui] trait UISparkListener extends SparkListener @@ -36,16 +37,20 @@ private[ui] trait UISparkListener extends SparkListener * * (1) If the UI is live, GatewayUISparkListener posts each event to all attached listeners * then logs it as JSON. This centralizes event logging and avoids having all attached - * listeners log the events on their own. By default, GatewayUISparkListener logs one - * file per job, though this needs not be the case. + * listeners log the events on their own. * * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized * from the event logs to all attached listeners. */ -private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends SparkListener { +private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) extends SparkListener { // Log events only if the UI is live - private val logger: Option[FileLogger] = if (live) Some(new FileLogger()) else None + private val logger: Option[FileLogger] = { + if (sc != null && sc.conf.getBoolean("spark.eventLog.enabled", false)) { + val logDir = sc.conf.get("spark.eventLog.dir", "/tmp/spark-events") + Some(new FileLogger(logDir)) + } else None + } // Children listeners for which this gateway is responsible private val listeners = ArrayBuffer[UISparkListener]() @@ -61,17 +66,14 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends } } - private def startLogger() = logger.foreach(_.start()) - private def closeLogger() = logger.foreach(_.close()) - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { listeners.foreach(_.onStageSubmitted(stageSubmitted)) - logEvent(stageSubmitted, flushLogger = true) + logEvent(stageSubmitted) } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { listeners.foreach(_.onStageCompleted(stageCompleted)) - logEvent(stageCompleted, flushLogger = true) + logEvent(stageCompleted) } override def onTaskStart(taskStart: SparkListenerTaskStart) { @@ -89,14 +91,12 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends override def onJobStart(jobStart: SparkListenerJobStart) { listeners.foreach(_.onJobStart(jobStart)) - startLogger() - logEvent(jobStart) + logEvent(jobStart, flushLogger = true) } override def onJobEnd(jobEnd: SparkListenerJobEnd) { listeners.foreach(_.onJobEnd(jobEnd)) - logEvent(jobEnd) - closeLogger() + logEvent(jobEnd, flushLogger = true) } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { @@ -118,10 +118,10 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, live: Boolean) extends override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { listeners.foreach(_.onUnpersistRDD(unpersistRDD)) - // In case logger has not already started, as unpersist may be called between jobs - startLogger() logEvent(unpersistRDD, flushLogger = true) } + + def stop() = logger.foreach(_.close()) } /** diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b71232d920c6c..de334096c00a3 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -29,8 +29,7 @@ import org.apache.spark.Logging * @param name An identifier of each FileLogger instance */ class FileLogger( - logDir: String = Option(System.getenv("SPARK_LOG_DIR")) - .getOrElse("/tmp/spark-%s".format(System.getProperty("user.name", "user"))), + logDir: String, name: String = String.valueOf(System.currentTimeMillis())) extends Logging { diff --git a/docs/configuration.md b/docs/configuration.md index 8e4c48c81f8be..602ef5deff15c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -444,7 +444,21 @@ Apart from these, the following properties are also available, and may be useful
    + + + + + + + + + + From cceff2b71bb0a8bfae7a47939edb97d5cc13effb Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Fri, 28 Feb 2014 21:55:11 -0800 Subject: [PATCH 41/68] Fix 100 char format fail --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 a3ab97d2b5aff..2e9d3a9cc7458 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -540,7 +540,8 @@ class DAGScheduler( stageIdToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } From 36b3e5da9be9c5cb0c29fca62a76ccd66fe01c1c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 2 Mar 2014 23:05:17 -0800 Subject: [PATCH 42/68] Add HDFS support for event logging --- .../org/apache/spark/ui/UISparkListener.scala | 3 +- .../org/apache/spark/util/FileLogger.scala | 48 ++++++++++++++----- 2 files changed, 37 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 4c10b0e98bb58..d0e5647318b2c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -48,7 +48,8 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) exte private val logger: Option[FileLogger] = { if (sc != null && sc.conf.getBoolean("spark.eventLog.enabled", false)) { val logDir = sc.conf.get("spark.eventLog.dir", "/tmp/spark-events") - Some(new FileLogger(logDir)) + val overwrite = sc.conf.getBoolean("spark.eventLog.overwrite", true) + Some(new FileLogger(logDir, overwriteExistingFiles = overwrite)) } else None } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index de334096c00a3..f5831e1382c54 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -19,32 +19,37 @@ package org.apache.spark.util import java.io._ import java.text.SimpleDateFormat +import java.net.URI import java.util.Date import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.fs.{Path, FileSystem} /** * A generic class for logging information to file - * @param logDir Path to the directory in which files are logged + * @param logBaseDir Path to the directory in which files are logged * @param name An identifier of each FileLogger instance + * @param overwriteExistingFiles Whether to overwrite existing files */ class FileLogger( - logDir: String, - name: String = String.valueOf(System.currentTimeMillis())) + logBaseDir: String, + name: String = String.valueOf(System.currentTimeMillis()), + overwriteExistingFiles: Boolean = true) extends Logging { - private val logPath = logDir.stripSuffix("/") + "/" + name + private val logDir = logBaseDir.stripSuffix("/") + "/" + name private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private var fileIndex = 0 private var writer: Option[PrintWriter] = { createLogDir() - Some(createWriter()) + createWriter() } /** Create a logging directory with the given path */ private def createLogDir() = { - val dir = new File(logPath) + val dir = new File(logDir) if (dir.exists) { logWarning("Logging directory already exists: " + logDir) } @@ -54,12 +59,29 @@ class FileLogger( } } - /** Create a new writer to the file identified with the given path */ - private def createWriter() = { - // Overwrite any existing file - val fileWriter = new FileWriter(logPath + "/" + fileIndex) - val bufferedWriter = new BufferedWriter(fileWriter) - new PrintWriter(bufferedWriter) + /** + * Create a new writer to the file identified with the given path file. + * File systems currently supported include hdfs, s3, and the local file system. + */ + private def createWriter(): Option[PrintWriter] = { + val logPath = logDir + "/" + fileIndex + val uri = new URI(logPath) + val fileStream = uri.getScheme match { + case "hdfs" | "s3" => + val conf = SparkHadoopUtil.get.newConfiguration() + val fs = FileSystem.get(uri, conf) + val path = new Path(logPath) + fs.create(path, overwriteExistingFiles) + case "file" | null => + // org.apache.hadoop.fs.FileSystem (r1.0.4) does not flush on local files + // Second parameter is whether to append + new FileOutputStream(logPath, !overwriteExistingFiles) + case _ => + logWarning("Given logging directory is invalid: %s".format(logDir)) + return None + } + val bufferedStream = new BufferedOutputStream(fileStream) + Some(new PrintWriter(bufferedStream)) } /** @@ -96,7 +118,7 @@ class FileLogger( def start() = { writer.getOrElse { fileIndex += 1 - writer = Some(createWriter()) + writer = createWriter() } } } From 03eda0b267f47c539c12c2ea990836db154bd8c5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 3 Mar 2014 16:14:18 -0800 Subject: [PATCH 43/68] Fix HDFS flush behavior org.apache.hadoop.fs.FSDataOutputStream only supports sync(), but not flush(). This means that flushing higher level streams doesn't actually do anything if the Hadoop FileSystem is used. This is now fixed. Further, this clarifies why logging local files is handled as a special case by referencing a known, unresolved bug in HDFS (HADOOP-7844). --- .../org/apache/spark/ui/UISparkListener.scala | 6 +- .../org/apache/spark/util/FileLogger.scala | 72 ++++++++++++++----- 2 files changed, 56 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index d0e5647318b2c..6c32a8752df39 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -22,11 +22,11 @@ import scala.collection.mutable.ArrayBuffer import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark.util.FileLogger import org.apache.spark.util.JsonProtocol -import org.apache.spark.SparkContext private[ui] trait UISparkListener extends SparkListener @@ -74,7 +74,7 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) exte override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { listeners.foreach(_.onStageCompleted(stageCompleted)) - logEvent(stageCompleted) + logEvent(stageCompleted, flushLogger = true) } override def onTaskStart(taskStart: SparkListenerTaskStart) { @@ -122,7 +122,7 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) exte logEvent(unpersistRDD, flushLogger = true) } - def stop() = logger.foreach(_.close()) + def stop() = logger.foreach(_.stop()) } /** diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f5831e1382c54..f7feed78d80a5 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -24,10 +24,11 @@ import java.util.Date import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.{FSDataOutputStream, Path, FileSystem} /** - * A generic class for logging information to file + * A generic class for logging information to file. + * * @param logBaseDir Path to the directory in which files are logged * @param name An identifier of each FileLogger instance * @param overwriteExistingFiles Whether to overwrite existing files @@ -42,13 +43,17 @@ class FileLogger( private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private var fileIndex = 0 + // Only defined if the file system scheme uses the Hadoop API + private var hadoopDataStream: Option[FSDataOutputStream] = None + private var hadoopFileSystem: Option[FileSystem] = None + private var writer: Option[PrintWriter] = { createLogDir() createWriter() } /** Create a logging directory with the given path */ - private def createLogDir() = { + private def createLogDir() { val dir = new File(logDir) if (dir.exists) { logWarning("Logging directory already exists: " + logDir) @@ -60,27 +65,37 @@ class FileLogger( } /** - * Create a new writer to the file identified with the given path file. - * File systems currently supported include hdfs, s3, and the local file system. + * Create a new writer for the file identified by the given path. + * + * File systems currently supported include HDFS, S3, and the local file system. + * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ private def createWriter(): Option[PrintWriter] = { val logPath = logDir + "/" + fileIndex val uri = new URI(logPath) - val fileStream = uri.getScheme match { + + val dataStream = uri.getScheme match { case "hdfs" | "s3" => - val conf = SparkHadoopUtil.get.newConfiguration() - val fs = FileSystem.get(uri, conf) + val fs = hadoopFileSystem.getOrElse { + val conf = SparkHadoopUtil.get.newConfiguration() + hadoopFileSystem = Some(FileSystem.get(uri, conf)) + hadoopFileSystem.get + } val path = new Path(logPath) - fs.create(path, overwriteExistingFiles) + hadoopDataStream = Some(fs.create(path, overwriteExistingFiles)) + hadoopDataStream.get + case "file" | null => - // org.apache.hadoop.fs.FileSystem (r1.0.4) does not flush on local files // Second parameter is whether to append new FileOutputStream(logPath, !overwriteExistingFiles) - case _ => - logWarning("Given logging directory is invalid: %s".format(logDir)) - return None + + case unsupportedScheme => + throw new UnsupportedOperationException("File system scheme %s is not supported!" + .format(unsupportedScheme)) } - val bufferedStream = new BufferedOutputStream(fileStream) + + val bufferedStream = new BufferedOutputStream(dataStream) Some(new PrintWriter(bufferedStream)) } @@ -89,7 +104,7 @@ class FileLogger( * @param msg The message to be logged * @param withTime Whether to prepend message with a timestamp */ - def log(msg: String, withTime: Boolean = false) = { + def log(msg: String, withTime: Boolean = false) { var writeInfo = msg if (withTime) { val date = new Date(System.currentTimeMillis()) @@ -105,20 +120,39 @@ class FileLogger( */ def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) - /** Flush the writer to disk manually */ - def flush() = writer.foreach(_.flush()) + /** + * Flush the writer to disk manually. + * + * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be + * sync()'ed manually as it does not support flush(), which is invoked by when higher + * level streams are flushed. + */ + def flush() { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.sync()) + } /** Close the writer. Any subsequent calls to log or flush will have no effect. */ - def close() = { + def close() { writer.foreach(_.close()) writer = None } /** Start a new writer (for a new file) if there does not already exist one */ - def start() = { + def start() { writer.getOrElse { fileIndex += 1 writer = createWriter() } } + + /** + * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger + * instance will throw exceptions. + */ + def stop() { + hadoopDataStream.foreach(_.close()) + hadoopFileSystem.foreach(_.close()) + writer.foreach(_.close()) + } } From aef411cfe1d8bbf3cdc53ae03b5cfb3401a660ba Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 3 Mar 2014 18:08:55 -0800 Subject: [PATCH 44/68] Fix bug: storage status was not reflected on UI in the local case This is because the executor ID from task events and that from the storage status list are different. The former is "localhost", but the latter is "." To be consistent, we will only use "." --- .../org/apache/spark/ui/UISparkListener.scala | 20 +++++++++++++++++-- .../apache/spark/ui/exec/ExecutorsUI.scala | 10 ---------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 6c32a8752df39..fb95ca5cdb5b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -41,10 +41,16 @@ private[ui] trait UISparkListener extends SparkListener * * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized * from the event logs to all attached listeners. + * + * Event logging is specified by three configurable parameters: + * + * spark.eventLog.enabled - Whether event logging is enabled. + * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.overwrite - Whether to overwrite any existing files. */ private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) extends SparkListener { - // Log events only if the UI is live + // Log events only if the UI is live and event logging is enabled private val logger: Option[FileLogger] = { if (sc != null && sc.conf.getBoolean("spark.eventLog.enabled", false)) { val logDir = sc.conf.get("spark.eventLog.dir", "/tmp/spark-events") @@ -154,7 +160,7 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val info = taskEnd.taskInfo if (info != null) { - val execId = info.executorId + val execId = formatExecutorId(info.executorId) val metrics = taskEnd.taskMetrics if (metrics != null) { val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq()) @@ -172,6 +178,16 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { storageStatusList = executorsStateChange.storageStatusList } + + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). This results in + * duplicate rows for the same executor. Thus, in this mode, we aggregate these two + * rows and use the executor ID of "" to be consistent. + */ + protected def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId + } } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index a838dee66e69a..854f0c8b70e43 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -203,14 +203,4 @@ private[ui] class ExecutorsListener extends StorageStatusSparkListener { super.onTaskEnd(taskEnd) } } - - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). This results in - * duplicate rows for the same executor. Thus, in this mode, we aggregate these two - * rows and use the executor ID of "" to be consistent. - */ - private def formatExecutorId(execId: String): String = { - if (execId == "localhost") "" else execId - } } From bb4c503597f291fd26bd7e26639416a6faf4488a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 3 Mar 2014 18:36:13 -0800 Subject: [PATCH 45/68] Use a more mnemonic path for logging Also get rid of a couple of unused vals. --- core/src/main/scala/org/apache/spark/ui/UISparkListener.scala | 3 ++- .../src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala | 2 -- core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala | 2 -- .../scala/org/apache/spark/ui/storage/BlockManagerUI.scala | 2 -- 4 files changed, 2 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index fb95ca5cdb5b2..4d109c270548c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -54,8 +54,9 @@ private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) exte private val logger: Option[FileLogger] = { if (sc != null && sc.conf.getBoolean("spark.eventLog.enabled", false)) { val logDir = sc.conf.get("spark.eventLog.dir", "/tmp/spark-events") + val appName = sc.appName.replaceAll("[ /]", "_").toLowerCase val overwrite = sc.conf.getBoolean("spark.eventLog.overwrite", true) - Some(new FileLogger(logDir, overwriteExistingFiles = overwrite)) + Some(new FileLogger(logDir, appName, overwriteExistingFiles = overwrite)) } else None } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index c536fde3efc4c..5c3b2216eaa20 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -31,8 +31,6 @@ import org.apache.spark.ui._ private[ui] class EnvironmentUI(parent: SparkUI) { lazy val appName = parent.appName lazy val listener = _listener.get - val live = parent.live - val sc = parent.sc private var _listener: Option[EnvironmentListener] = None diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 854f0c8b70e43..f9a0155afa5af 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -34,8 +34,6 @@ import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { lazy val appName = parent.appName lazy val listener = _listener.get - val live = parent.live - val sc = parent.sc private var _listener: Option[ExecutorsListener] = None diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 4b59c9609045e..73d1dc73066ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -28,8 +28,6 @@ import org.apache.spark.ui._ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val appName = parent.appName lazy val listener = _listener.get - val live = parent.live - val sc = parent.sc private val indexPage = new IndexPage(this) private val rddPage = new RDDPage(this) From 18b256d37c7a6335c002759a65d2df36dc8faf2e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Mar 2014 16:27:35 -0800 Subject: [PATCH 46/68] Refactor out event logging and replaying logic from UI This involves taking apart what was once GatewayUISparkListener, and introducing in its place the EventLoggingListener and SparkReplayerBus (and AppNameListener, but this one is not important). This allows the event logging and replaying functionalities to be used outside of the context of the SparkUI. This commit also ensures all file system modifications go through the Hadoop FileSystem. This adds the functionality of reading event logs from HDFS. This also fixes a bug in which Spark would attempt to create a log directory in the local file system when the path in fact refers to an HDFS directory. --- .../org/apache/spark/scheduler/EventBus.scala | 79 ++++++++++++ .../scheduler/EventLoggingListener.scala | 79 ++++++++++++ .../spark/scheduler/SparkListenerBus.scala | 46 +------ .../spark/scheduler/SparkReplayerBus.scala | 92 ++++++++++++++ .../scala/org/apache/spark/ui/SparkUI.scala | 115 ++++++------------ .../org/apache/spark/ui/UISparkListener.scala | 110 +---------------- .../apache/spark/ui/env/EnvironmentUI.scala | 3 - .../apache/spark/ui/exec/ExecutorsUI.scala | 2 - .../apache/spark/ui/jobs/JobProgressUI.scala | 2 - .../spark/ui/storage/BlockManagerUI.scala | 2 - .../org/apache/spark/util/FileLogger.scala | 61 +++++----- .../scala/org/apache/spark/util/Utils.scala | 31 ++++- 12 files changed, 353 insertions(+), 269 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/EventBus.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala new file mode 100644 index 0000000000000..529ee1a8d09ca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +/** + * A SparkListenerEvent bus that relays events to its listeners. + */ +private[spark] trait EventBus { + + // SparkListeners attached to this event bus + protected val sparkListeners = new ArrayBuffer[SparkListener] + with mutable.SynchronizedBuffer[SparkListener] + + def addListener(listener: SparkListener) { + sparkListeners += listener + } + + /** + * Post an event to all attached listeners. Return true if the shutdown event is posted. + */ + protected def postToAll(event: SparkListenerEvent): Boolean = { + postToListeners(event, sparkListeners) + } + + /** + * Post an event to a given list of listeners. Return true if the shutdown event is posted. + */ + protected def postToListeners( + event: SparkListenerEvent, + listeners: Seq[SparkListener]): Boolean = { + + event match { + case stageSubmitted: SparkListenerStageSubmitted => + listeners.foreach(_.onStageSubmitted(stageSubmitted)) + case stageCompleted: SparkListenerStageCompleted => + listeners.foreach(_.onStageCompleted(stageCompleted)) + case jobStart: SparkListenerJobStart => + listeners.foreach(_.onJobStart(jobStart)) + case jobEnd: SparkListenerJobEnd => + listeners.foreach(_.onJobEnd(jobEnd)) + case taskStart: SparkListenerTaskStart => + listeners.foreach(_.onTaskStart(taskStart)) + case taskGettingResult: SparkListenerTaskGettingResult => + listeners.foreach(_.onTaskGettingResult(taskGettingResult)) + case taskEnd: SparkListenerTaskEnd => + listeners.foreach(_.onTaskEnd(taskEnd)) + case applicationStart: SparkListenerApplicationStart => + listeners.foreach(_.onApplicationStart(applicationStart)) + case environmentUpdate: SparkListenerEnvironmentUpdate => + listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case executorsStateChange: SparkListenerExecutorsStateChange => + listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) + case unpersistRDD: SparkListenerUnpersistRDD => + listeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case SparkListenerShutdown => + return true + case _ => + } + false + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala new file mode 100644 index 0000000000000..6ec7f8b2a5320 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.{Logging, SparkConf} + +/** + * A SparkListener that logs events to persistent storage. + * + * Event logging is specified by three configurable parameters: + * spark.eventLog.enabled - Whether event logging is enabled. + * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.overwrite - Whether to overwrite any existing files. + */ +private[spark] class EventLoggingListener(appName: String, conf: SparkConf) + extends SparkListener with Logging { + + private val shouldLog = conf.getBoolean("spark.eventLog.enabled", false) + private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) + private val logDir = conf.get("spark.eventLog.dir", "/tmp/spark-events") + private val name = appName.replaceAll("[ /]", "-").toLowerCase + "-" + System.currentTimeMillis() + + private val logger: Option[FileLogger] = if (shouldLog) { + logInfo("Logging events to %s".format(logDir)) + Some(new FileLogger(logDir, name, overwrite = shouldOverwrite)) + } else { + logWarning("Event logging is disabled. To enable it, set spark.eventLog.enabled to true.") + None + } + + /** Log the event as JSON */ + private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { + val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) + logger.foreach(_.logLine(eventJson)) + if (flushLogger) { + logger.foreach(_.flush()) + } + } + + // Events that do not trigger a flush + override def onStageSubmitted(event: SparkListenerStageSubmitted) = logEvent(event) + override def onTaskStart(event: SparkListenerTaskStart) = logEvent(event) + override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = logEvent(event) + override def onTaskEnd(event: SparkListenerTaskEnd) = logEvent(event) + override def onApplicationStart(event: SparkListenerApplicationStart) = logEvent(event) + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = logEvent(event) + + // Events that trigger a flush + override def onStageCompleted(event: SparkListenerStageCompleted) = + logEvent(event, flushLogger = true) + override def onJobStart(event: SparkListenerJobStart) = + logEvent(event, flushLogger = true) + override def onJobEnd(event: SparkListenerJobEnd) = + logEvent(event, flushLogger = true) + override def onExecutorsStateChange(event: SparkListenerExecutorsStateChange) = + logEvent(event, flushLogger = true) + override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = + logEvent(event, flushLogger = true) + + def stop() = logger.foreach(_.stop()) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 306f5fc5b610c..f6cb3702f8bef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -19,13 +19,10 @@ package org.apache.spark.scheduler import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} - import org.apache.spark.Logging /** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ -private[spark] class SparkListenerBus extends Logging { - private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener] +private[spark] class SparkListenerBus extends EventBus with Logging { /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ @@ -40,7 +37,7 @@ private[spark] class SparkListenerBus extends Logging { override def run() { while (true) { val event = eventQueue.take - val shutdown = postToListeners(event, sparkListeners) + val shutdown = postToAll(event) if (shutdown) { // Get out of the while loop and shutdown the daemon thread return @@ -49,45 +46,6 @@ private[spark] class SparkListenerBus extends Logging { } }.start() - /** - * Post an event to a given list of listeners. Return true if the shutdown event is posted. - */ - private[spark] def postToListeners(event: SparkListenerEvent, listeners: Seq[SparkListener]) - : Boolean = { - event match { - case stageSubmitted: SparkListenerStageSubmitted => - listeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: SparkListenerStageCompleted => - listeners.foreach(_.onStageCompleted(stageCompleted)) - case jobStart: SparkListenerJobStart => - listeners.foreach(_.onJobStart(jobStart)) - case jobEnd: SparkListenerJobEnd => - listeners.foreach(_.onJobEnd(jobEnd)) - case taskStart: SparkListenerTaskStart => - listeners.foreach(_.onTaskStart(taskStart)) - case taskGettingResult: SparkListenerTaskGettingResult => - listeners.foreach(_.onTaskGettingResult(taskGettingResult)) - case taskEnd: SparkListenerTaskEnd => - listeners.foreach(_.onTaskEnd(taskEnd)) - case applicationStart: SparkListenerApplicationStart => - listeners.foreach(_.onApplicationStart(applicationStart)) - case environmentUpdate: SparkListenerEnvironmentUpdate => - listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) - case executorsStateChange: SparkListenerExecutorsStateChange => - listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) - case unpersistRDD: SparkListenerUnpersistRDD => - listeners.foreach(_.onUnpersistRDD(unpersistRDD)) - case SparkListenerShutdown => - return true - case _ => - } - false - } - - def addListener(listener: SparkListener) { - sparkListeners += listener - } - def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) if (!eventAdded && !queueFullErrorMessageLogged) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala new file mode 100644 index 0000000000000..d30f5ff112cd1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.io.InputStream + +import scala.io.Source + +import it.unimi.dsi.fastutil.io.FastBufferedInputStream +import org.apache.hadoop.fs.{Path, FileSystem} +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.Logging +import org.apache.spark.util.{Utils, JsonProtocol} + +/** + * An EventBus that replays logged events from persisted storage. + */ +private[spark] class SparkReplayerBus extends EventBus with Logging { + + /** + * Return a list of paths representing log files in the given directory. + */ + private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { + val path = new Path(logDir) + if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { + logWarning("Log path provided is not a valid directory: %s".format(logDir)) + return Array[Path]() + } + val logStatus = fileSystem.listStatus(path) + if (logStatus == null || !logStatus.exists(!_.isDir)) { + logWarning("Log path provided contains no log files: %s".format(logDir)) + return Array[Path]() + } + logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + } + + /** + * Replay each event in the order maintained in the given logs. + */ + def replay(logDir: String): Boolean = { + val fileSystem = Utils.getHadoopFileSystem(logDir) + val logPaths = getLogFilePaths(logDir, fileSystem) + if (logPaths.length == 0) { + return false + } + + logPaths.foreach { path => + // In case there is an exception, keep track of the highest level stream to close it later + var streamToClose: Option[InputStream] = None + var currentLine = "" + try { + val fstream = fileSystem.open(path) + val bstream = new FastBufferedInputStream(fstream) + streamToClose = Some(bstream) + + // Parse each line as an event and post it to all attached listeners + val lines = Source.fromInputStream(bstream).getLines() + lines.foreach { line => + currentLine = line + val event = JsonProtocol.sparkEventFromJson(parse(line)) + postToAll(event) + } + } catch { + case e: Exception => + logWarning("Exception in parsing UI logs for %s".format(path)) + logWarning(currentLine + "\n") + logDebug(e.getMessage + e.getStackTraceString) + } finally { + streamToClose.foreach(_.close()) + } + } + fileSystem.close() + true + } + +} 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 c9d3251cc30da..f231072ed9203 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,23 +17,16 @@ package org.apache.spark.ui -import java.io.{FileInputStream, File} - -import scala.io.Source - -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.eclipse.jetty.server.{Handler, Server} -import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext, SparkEnv} -import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.{SparkReplayerBus, EventLoggingListener} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils -import org.apache.spark.util.JsonProtocol /** Top level user interface for Spark. */ private[spark] class SparkUI(val sc: SparkContext) extends Logging { @@ -51,15 +44,16 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { var started = false var appName = "" - private val handlers = Seq[(String, Handler)]( - ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("/", createRedirectHandler("/stages")) - ) private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) private val env = new EnvironmentUI(this) private val exec = new ExecutorsUI(this) + private val handlers = Seq[(String, Handler)]( + ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), + ("/", createRedirectHandler("/stages")) + ) + // Add MetricsServlet handlers by default private val metricsServletHandlers = if (live) { SparkEnv.get.metricsSystem.getServletHandlers @@ -70,14 +64,15 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers - // Maintain a gateway listener for all events to simplify event logging - private var _gatewayListener: Option[GatewayUISparkListener] = None - def gatewayListener = _gatewayListener.getOrElse { - val gateway = new GatewayUISparkListener(this, sc) - _gatewayListener = Some(gateway) - gateway - } + // A simple listener that sets the app name for this SparkUI + private val appNameListener = new AppNameListener(this) + + // Only log events if this SparkUI is live + private var eventLogger: Option[EventLoggingListener] = None + + // Only replay events if this SparkUI is not live + private var replayerBus: Option[SparkReplayerBus] = None // Only meaningful if port is set before binding def setPort(p: Int) = { @@ -111,22 +106,26 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { // DAGScheduler() requires that the port of this server is known // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize - - if (live) { - // Listen for new events only if this UI is live - sc.addSparkListener(gatewayListener) - } storage.start() jobs.start() env.start() exec.start() - started = true - } - def stop() { - server.foreach(_.stop()) - _gatewayListener.foreach(_.stop()) - logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + // Listen for events from the SparkContext if it exists, otherwise from persisted storage + val eventBus = if (live) { + eventLogger = Some(new EventLoggingListener(sc.appName, sc.conf)) + sc.listenerBus.addListener(eventLogger.get) + sc.listenerBus + } else { + replayerBus = Some(new SparkReplayerBus) + replayerBus.get + } + eventBus.addListener(storage.listener) + eventBus.addListener(jobs.listener) + eventBus.addListener(env.listener) + eventBus.addListener(exec.listener) + eventBus.addListener(appNameListener) + started = true } /** @@ -135,56 +134,16 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { * This method must be invoked after the SparkUI has started. Return true if log files * are found and processed. */ - def renderFromPersistedStorage(dirPath: String): Boolean = { - assert(!live, "Live Spark Web UI attempted to render from persisted storage!") - assert(started, "Spark Web UI attempted to render from persisted storage before starting!") - - // Check validity of the given path - val logDir = new File(dirPath) - if (!logDir.exists || !logDir.isDirectory) { - logWarning("Given invalid log path %s when rendering persisted Spark Web UI!" - .format(dirPath)) - return false - } - // Assume events are ordered not only within each log file, but also across files by file name - val logFiles = logDir.listFiles.filter(_.isFile).sortBy(_.getName) - if (logFiles.size == 0) { - logWarning("No logs found in given directory %s when rendering persisted Spark Web UI!" - .format(dirPath)) - return false - } - - // Replay events in each event log - // Use a new SparkListenerBus to avoid depending on SparkContext - val bus = new SparkListenerBus - logFiles.foreach { file => processEventLog(file, bus) } - true + def renderFromPersistedStorage(logDir: String): Boolean = { + assume(!live, "Live Spark Web UI attempted to render from persisted storage!") + assume(started, "Spark Web UI attempted to render from persisted storage before starting!") + replayerBus.get.replay(logDir) } - /** - * Replay each event in the order maintained in the given log to the gateway listener. - * - * A custom SparkListenerBus, rather than the DAG scheduler's, is used to decouple the - * replaying of logged events from the creation of a SparkContext. - */ - private def processEventLog(file: File, listenerBus: SparkListenerBus) = { - val fileStream = new FileInputStream(file) - val bufferedStream = new FastBufferedInputStream(fileStream) - var currentLine = "" - try { - val lines = Source.fromInputStream(bufferedStream).getLines() - lines.foreach { line => - currentLine = line - val event = JsonProtocol.sparkEventFromJson(parse(line)) - listenerBus.postToListeners(event, Seq(gatewayListener)) - } - } catch { - case e: Exception => - logWarning("Exception in parsing UI logs for %s".format(file.getAbsolutePath)) - logWarning(currentLine + "\n") - logDebug(e.getMessage + e.getStackTraceString) - } - bufferedStream.close() + def stop() { + server.foreach(_.stop()) + eventLogger.foreach(_.stop()) + logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 4d109c270548c..c696f0275a9c2 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -18,122 +18,24 @@ package org.apache.spark.ui import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import org.json4s.jackson.JsonMethods._ - -import org.apache.spark.SparkContext import org.apache.spark.scheduler._ import org.apache.spark.storage._ -import org.apache.spark.util.FileLogger -import org.apache.spark.util.JsonProtocol private[ui] trait UISparkListener extends SparkListener /** - * A SparkListener that serves as an entry point for all events posted to the UI. - * - * GatewayUISparkListener achieves two functions: - * - * (1) If the UI is live, GatewayUISparkListener posts each event to all attached listeners - * then logs it as JSON. This centralizes event logging and avoids having all attached - * listeners log the events on their own. - * - * (2) If the UI is rendered from disk, GatewayUISparkListener replays each event deserialized - * from the event logs to all attached listeners. - * - * Event logging is specified by three configurable parameters: - * - * spark.eventLog.enabled - Whether event logging is enabled. - * spark.eventLog.dir - Path to the directory in which events are logged. - * spark.eventLog.overwrite - Whether to overwrite any existing files. + * A SparkListener that listens only for application start events to set the app name for the UI. */ -private[ui] class GatewayUISparkListener(parent: SparkUI, sc: SparkContext) extends SparkListener { - - // Log events only if the UI is live and event logging is enabled - private val logger: Option[FileLogger] = { - if (sc != null && sc.conf.getBoolean("spark.eventLog.enabled", false)) { - val logDir = sc.conf.get("spark.eventLog.dir", "/tmp/spark-events") - val appName = sc.appName.replaceAll("[ /]", "_").toLowerCase - val overwrite = sc.conf.getBoolean("spark.eventLog.overwrite", true) - Some(new FileLogger(logDir, appName, overwriteExistingFiles = overwrite)) - } else None - } - - // Children listeners for which this gateway is responsible - private val listeners = ArrayBuffer[UISparkListener]() - - def registerSparkListener(listener: UISparkListener) = listeners += listener - - /** Log the event as JSON */ - private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { - val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) - logger.foreach(_.logLine(eventJson)) - if (flushLogger) { - logger.foreach(_.flush()) - } - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - listeners.foreach(_.onStageSubmitted(stageSubmitted)) - logEvent(stageSubmitted) - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - listeners.foreach(_.onStageCompleted(stageCompleted)) - logEvent(stageCompleted, flushLogger = true) - } - - override def onTaskStart(taskStart: SparkListenerTaskStart) { - listeners.foreach(_.onTaskStart(taskStart)) - logEvent(taskStart) - } - override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { - listeners.foreach(_.onTaskGettingResult(taskGettingResult)) - logEvent(taskGettingResult) - } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - listeners.foreach(_.onTaskEnd(taskEnd)) - logEvent(taskEnd) - } - - override def onJobStart(jobStart: SparkListenerJobStart) { - listeners.foreach(_.onJobStart(jobStart)) - logEvent(jobStart, flushLogger = true) - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - listeners.foreach(_.onJobEnd(jobEnd)) - logEvent(jobEnd, flushLogger = true) - } - +private[ui] class AppNameListener(parent: SparkUI) extends UISparkListener { override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - // For live UI's, this should be equivalent to sc.appName - parent.setAppName(applicationStart.appName) - listeners.foreach(_.onApplicationStart(applicationStart)) - logEvent(applicationStart, flushLogger = true) + val appName = applicationStart.appName + parent.setAppName(appName) } - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) - logEvent(environmentUpdate) - } - - override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { - listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) - logEvent(executorsStateChange, flushLogger = true) - } - - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { - listeners.foreach(_.onUnpersistRDD(unpersistRDD)) - logEvent(unpersistRDD, flushLogger = true) - } - - def stop() = logger.foreach(_.stop()) } /** - * A UISparkListener that maintains executor storage status + * A SparkListener that maintains executor storage status */ private[ui] class StorageStatusSparkListener extends UISparkListener { var storageStatusList = Seq[StorageStatus]() @@ -192,7 +94,7 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } /** - * A UISparkListener that maintains RDD information + * A SparkListener that maintains RDD information */ private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 5c3b2216eaa20..7476397fb0afc 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -20,7 +20,6 @@ package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest import scala.xml.Node - import org.eclipse.jetty.server.Handler import org.apache.spark.scheduler._ @@ -35,9 +34,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) { private var _listener: Option[EnvironmentListener] = None def start() { - val gateway = parent.gatewayListener _listener = Some(new EnvironmentListener()) - gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index f9a0155afa5af..9914b35c277b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -38,9 +38,7 @@ private[ui] class ExecutorsUI(parent: SparkUI) { private var _listener: Option[ExecutorsListener] = None def start() { - val gateway = parent.gatewayListener _listener = Some(new ExecutorsListener()) - gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 64de63f7aae05..09b820c765149 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -42,9 +42,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { private var _listener: Option[JobProgressListener] = None def start() { - val gateway = parent.gatewayListener _listener = Some(new JobProgressListener(sc, live)) - gateway.registerSparkListener(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 73d1dc73066ca..e128fff17e57c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -34,9 +34,7 @@ private[ui] class BlockManagerUI(parent: SparkUI) { private var _listener: Option[BlockManagerListener] = None def start() { - val gateway = parent.gatewayListener _listener = Some(new BlockManagerListener) - gateway.registerSparkListener(listener) } def getHandlers = Seq[(String, Handler)]( diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f7feed78d80a5..b0f9e835bf681 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -22,81 +22,82 @@ import java.text.SimpleDateFormat import java.net.URI import java.util.Date +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream + import org.apache.spark.Logging -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.fs.{FSDataOutputStream, Path, FileSystem} +import org.apache.hadoop.fs.{FSDataOutputStream, Path} /** * A generic class for logging information to file. * * @param logBaseDir Path to the directory in which files are logged * @param name An identifier of each FileLogger instance - * @param overwriteExistingFiles Whether to overwrite existing files + * @param overwrite Whether to overwrite existing files */ class FileLogger( logBaseDir: String, name: String = String.valueOf(System.currentTimeMillis()), - overwriteExistingFiles: Boolean = true) + overwrite: Boolean = true) extends Logging { private val logDir = logBaseDir.stripSuffix("/") + "/" + name private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val fileSystem = Utils.getHadoopFileSystem(logDir) private var fileIndex = 0 - // Only defined if the file system scheme uses the Hadoop API + // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var hadoopFileSystem: Option[FileSystem] = None private var writer: Option[PrintWriter] = { createLogDir() - createWriter() + Some(createWriter()) } - /** Create a logging directory with the given path */ + /** Create a logging directory with the given path. */ private def createLogDir() { - val dir = new File(logDir) - if (dir.exists) { - logWarning("Logging directory already exists: " + logDir) + val path = new Path(logDir) + if (fileSystem.exists(path)) { + logWarning("Log directory already exists.") + if (overwrite) { + // Second parameter is whether to delete recursively + fileSystem.delete(path, true) + } } - if (!dir.exists && !dir.mkdirs()) { + if (!fileSystem.mkdirs(path)) { // Logger should throw a exception rather than continue to construct this object throw new IOException("Error in creating log directory:" + logDir) } } /** - * Create a new writer for the file identified by the given path. - * - * File systems currently supported include HDFS, S3, and the local file system. - * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. + * Create a new writer for the file identified by the given path. File systems currently + * supported include HDFS, S3, and the local file system. */ - private def createWriter(): Option[PrintWriter] = { + private def createWriter(): PrintWriter = { val logPath = logDir + "/" + fileIndex val uri = new URI(logPath) + /** + * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. + */ val dataStream = uri.getScheme match { case "hdfs" | "s3" => - val fs = hadoopFileSystem.getOrElse { - val conf = SparkHadoopUtil.get.newConfiguration() - hadoopFileSystem = Some(FileSystem.get(uri, conf)) - hadoopFileSystem.get - } val path = new Path(logPath) - hadoopDataStream = Some(fs.create(path, overwriteExistingFiles)) + hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get case "file" | null => // Second parameter is whether to append - new FileOutputStream(logPath, !overwriteExistingFiles) + new FileOutputStream(logPath, !overwrite) case unsupportedScheme => throw new UnsupportedOperationException("File system scheme %s is not supported!" .format(unsupportedScheme)) } - val bufferedStream = new BufferedOutputStream(dataStream) - Some(new PrintWriter(bufferedStream)) + val bufferedStream = new FastBufferedOutputStream(dataStream) + new PrintWriter(bufferedStream) } /** @@ -138,11 +139,11 @@ class FileLogger( writer = None } - /** Start a new writer (for a new file) if there does not already exist one */ + /** Start a writer for a new file if one does not already exit */ def start() { writer.getOrElse { fileIndex += 1 - writer = createWriter() + writer = Some(createWriter()) } } @@ -152,7 +153,7 @@ class FileLogger( */ def stop() { hadoopDataStream.foreach(_.close()) - hadoopFileSystem.foreach(_.close()) writer.foreach(_.close()) + fileSystem.close() } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 55eee311a5da1..d61b2b3e4ff55 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -297,8 +297,7 @@ private[spark] object Utils extends Logging { } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others - val conf = SparkHadoopUtil.get.newConfiguration() - val fs = FileSystem.get(uri, conf) + val fs = getHadoopFileSystem(uri) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) @@ -469,7 +468,7 @@ private[spark] object Utils extends Logging { * millisecond. */ def getUsedTimeMs(startTimeMs: Long): String = { - return " " + (System.currentTimeMillis - startTimeMs) + " ms" + " " + (System.currentTimeMillis - startTimeMs) + " ms" } /** @@ -760,7 +759,7 @@ private[spark] object Utils extends Logging { } var i = 0 while (i < s.length) { - var nextChar = s.charAt(i) + val nextChar = s.charAt(i) if (inDoubleQuote) { if (nextChar == '"') { inDoubleQuote = false @@ -880,4 +879,28 @@ private[spark] object Utils extends Logging { } def emptyJson = JObject(List[JField]()) + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + * File systems currently supported include HDFS, S3, and the local file system. + */ + def getHadoopFileSystem(path: URI): FileSystem = { + path.getScheme match { + case "file" | "hdfs" | "s3" | null => + val conf = SparkHadoopUtil.get.newConfiguration() + FileSystem.get(path, conf) + case unsupportedScheme => + throw new UnsupportedOperationException("File system scheme %s is not supported!" + .format(unsupportedScheme)) + } + } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + * File systems currently supported include HDFS, S3, and the local file system. + */ + def getHadoopFileSystem(path: String): FileSystem = { + val uri = new URI(path) + getHadoopFileSystem(uri) + } } From e3754310b165609251251c3da8dc34478e8ad55b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Mar 2014 17:45:56 -0800 Subject: [PATCH 47/68] Add new constructors for SparkUI For two reasons - first, the existing way is ugly because we have to instantiate the SparkUI by calling new SparkUI(null). Second, this provides a way to configure the persisted port through SparkConf. --- .../scala/org/apache/spark/ui/SparkUI.scala | 24 +++++++------------ .../org/apache/spark/ui/UIReloader.scala | 11 ++++++--- 2 files changed, 17 insertions(+), 18 deletions(-) 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 f231072ed9203..d23d5cac14946 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, SparkContext, SparkEnv} +import org.apache.spark.{SparkConf, Logging, SparkContext, SparkEnv} import org.apache.spark.scheduler.{SparkReplayerBus, EventLoggingListener} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI @@ -29,15 +29,19 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark. */ -private[spark] class SparkUI(val sc: SparkContext) extends Logging { +private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logging { + + def this() = this(null, new SparkConf()) + def this(conf: SparkConf) = this(null, conf) + def this(sc: SparkContext) = this(sc, sc.conf) // If SparkContext is not provided, assume this UI is rendered from persisted storage val live = sc != null val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) var port = if (live) { - sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt } else { - SparkUI.DEFAULT_PERSISTED_PORT.toInt + conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt } var boundPort: Option[Int] = None var server: Option[Server] = None @@ -74,16 +78,6 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { // Only replay events if this SparkUI is not live private var replayerBus: Option[SparkReplayerBus] = None - // Only meaningful if port is set before binding - def setPort(p: Int) = { - if (boundPort.isDefined) { - logWarning("Attempted to set Spark Web UI port after it is already bound to %s." - .format(appUIAddress)) - } else { - port = p - } - } - def setAppName(name: String) = appName = name /** Bind the HTTP server which backs this web interface */ @@ -113,7 +107,7 @@ private[spark] class SparkUI(val sc: SparkContext) extends Logging { // Listen for events from the SparkContext if it exists, otherwise from persisted storage val eventBus = if (live) { - eventLogger = Some(new EventLoggingListener(sc.appName, sc.conf)) + eventLogger = Some(new EventLoggingListener(sc.appName, conf)) sc.listenerBus.addListener(eventLogger.get) sc.listenerBus } else { diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala index c6199967e6ffc..01de077782fc9 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -17,17 +17,22 @@ package org.apache.spark.ui +import org.apache.spark.SparkConf + /** - * Reload a persisted UI independently from a SparkContext + * A simple example that reloads a persisted UI independently from a SparkContext */ object UIReloader { def main(args: Array[String]) { if (args.length < 1) { - println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path]") + println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path] [port]") System.exit(1) } - val ui = new SparkUI(null) + val port = if (args.length == 2) args(1) else "14040" + val conf = new SparkConf() + conf.set("spark.persisted.ui.port", port) + val ui = new SparkUI(conf) ui.bind() ui.start() val success = ui.renderFromPersistedStorage(args(0)) From 1ba34070f4d2e463e2aa37e0d05c518bd8771da5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 4 Mar 2014 18:39:47 -0800 Subject: [PATCH 48/68] Add a few configurable options to event logging This includes compression and output buffer size. --- .../scheduler/EventLoggingListener.scala | 12 +++++++---- .../spark/scheduler/SparkReplayerBus.scala | 15 +++++++++----- .../scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../org/apache/spark/util/FileLogger.scala | 20 ++++++++++++++----- docs/configuration.md | 18 ++++++++++++++++- 5 files changed, 51 insertions(+), 16 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 6ec7f8b2a5320..ab671ba8f3f20 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -25,22 +25,26 @@ import org.apache.spark.{Logging, SparkConf} /** * A SparkListener that logs events to persistent storage. * - * Event logging is specified by three configurable parameters: + * Event logging is specified by the following configurable parameters: * spark.eventLog.enabled - Whether event logging is enabled. - * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.compress - Whether to compress logged events * spark.eventLog.overwrite - Whether to overwrite any existing files. + * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { private val shouldLog = conf.getBoolean("spark.eventLog.enabled", false) - private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) + private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", true) private val logDir = conf.get("spark.eventLog.dir", "/tmp/spark-events") + private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val name = appName.replaceAll("[ /]", "-").toLowerCase + "-" + System.currentTimeMillis() private val logger: Option[FileLogger] = if (shouldLog) { logInfo("Logging events to %s".format(logDir)) - Some(new FileLogger(logDir, name, overwrite = shouldOverwrite)) + Some(new FileLogger(logDir, name, conf, outputBufferSize, shouldCompress, shouldOverwrite)) } else { logWarning("Event logging is disabled. To enable it, set spark.eventLog.enabled to true.") None diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala index d30f5ff112cd1..f17a87ec2de8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala @@ -25,13 +25,18 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.util.{Utils, JsonProtocol} +import org.apache.spark.io.CompressionCodec /** * An EventBus that replays logged events from persisted storage. */ -private[spark] class SparkReplayerBus extends EventBus with Logging { +private[spark] class SparkReplayerBus(conf: SparkConf) extends EventBus with Logging { + private val compressed = conf.getBoolean("spark.eventLog.compress", false) + + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) /** * Return a list of paths representing log files in the given directory. @@ -67,10 +72,11 @@ private[spark] class SparkReplayerBus extends EventBus with Logging { try { val fstream = fileSystem.open(path) val bstream = new FastBufferedInputStream(fstream) - streamToClose = Some(bstream) + val cstream = if (compressed) compressionCodec.compressedInputStream(bstream) else bstream + streamToClose = Some(cstream) // Parse each line as an event and post it to all attached listeners - val lines = Source.fromInputStream(bstream).getLines() + val lines = Source.fromInputStream(cstream).getLines() lines.foreach { line => currentLine = line val event = JsonProtocol.sparkEventFromJson(parse(line)) @@ -88,5 +94,4 @@ private[spark] class SparkReplayerBus extends EventBus with Logging { fileSystem.close() true } - } 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 d23d5cac14946..5f5dd24c4f2a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -111,7 +111,7 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg sc.listenerBus.addListener(eventLogger.get) sc.listenerBus } else { - replayerBus = Some(new SparkReplayerBus) + replayerBus = Some(new SparkReplayerBus(conf)) replayerBus.get } eventBus.addListener(storage.listener) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b0f9e835bf681..6963ed9f09335 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -23,20 +23,26 @@ import java.net.URI import java.util.Date import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - -import org.apache.spark.Logging import org.apache.hadoop.fs.{FSDataOutputStream, Path} +import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.io.CompressionCodec + /** * A generic class for logging information to file. * * @param logBaseDir Path to the directory in which files are logged * @param name An identifier of each FileLogger instance + * @param outputBufferSize The buffer size to use when writing to an output stream in bytes + * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ class FileLogger( logBaseDir: String, name: String = String.valueOf(System.currentTimeMillis()), + conf: SparkConf = new SparkConf(), + outputBufferSize: Int = 8 * 1024, + compress: Boolean = false, overwrite: Boolean = true) extends Logging { @@ -45,6 +51,9 @@ class FileLogger( private val fileSystem = Utils.getHadoopFileSystem(logDir) private var fileIndex = 0 + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) + // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -81,7 +90,7 @@ class FileLogger( * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ - val dataStream = uri.getScheme match { + val dstream = uri.getScheme match { case "hdfs" | "s3" => val path = new Path(logPath) hadoopDataStream = Some(fileSystem.create(path, overwrite)) @@ -96,8 +105,9 @@ class FileLogger( .format(unsupportedScheme)) } - val bufferedStream = new FastBufferedOutputStream(dataStream) - new PrintWriter(bufferedStream) + val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream + new PrintWriter(cstream) } /** diff --git a/docs/configuration.md b/docs/configuration.md index 602ef5deff15c..40cc905f4042e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -140,6 +140,13 @@ Apart from these, the following properties are also available, and may be useful Port for your application's dashboard, which shows memory and workload data + + + + + @@ -454,11 +461,20 @@ Apart from these, the following properties are also available, and may be useful Whether to log spark events, useful for reconstructing the Web UI after the application has finished. + + + + + From 291b2be0f663643121b462c5d6dcda101943b2e0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Mar 2014 10:49:27 -0800 Subject: [PATCH 49/68] Correct directory in log message "INFO: Logging events to " --- .../org/apache/spark/scheduler/EventLoggingListener.scala | 5 +++-- core/src/main/scala/org/apache/spark/util/FileLogger.scala | 7 ++----- 2 files changed, 5 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 ab671ba8f3f20..52a7c9f6dfc97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -38,13 +38,14 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val shouldLog = conf.getBoolean("spark.eventLog.enabled", false) private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", true) - private val logDir = conf.get("spark.eventLog.dir", "/tmp/spark-events") private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 + private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") private val name = appName.replaceAll("[ /]", "-").toLowerCase + "-" + System.currentTimeMillis() + private val logDir = logBaseDir + "/" + name private val logger: Option[FileLogger] = if (shouldLog) { logInfo("Logging events to %s".format(logDir)) - Some(new FileLogger(logDir, name, conf, outputBufferSize, shouldCompress, shouldOverwrite)) + Some(new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite)) } else { logWarning("Event logging is disabled. To enable it, set spark.eventLog.enabled to true.") None diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 6963ed9f09335..1eaf5e6ecec00 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -31,22 +31,19 @@ import org.apache.spark.io.CompressionCodec /** * A generic class for logging information to file. * - * @param logBaseDir Path to the directory in which files are logged - * @param name An identifier of each FileLogger instance + * @param logDir Path to the directory in which files are logged * @param outputBufferSize The buffer size to use when writing to an output stream in bytes * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ class FileLogger( - logBaseDir: String, - name: String = String.valueOf(System.currentTimeMillis()), + logDir: String, conf: SparkConf = new SparkConf(), outputBufferSize: Int = 8 * 1024, compress: Boolean = false, overwrite: Boolean = true) extends Logging { - private val logDir = logBaseDir.stripSuffix("/") + "/" + name private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val fileSystem = Utils.getHadoopFileSystem(logDir) private var fileIndex = 0 From 4f69c4a3cb61582eb5b82f1f341446067b97afa2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Mar 2014 15:43:34 -0800 Subject: [PATCH 50/68] Master UI - Rebuild SparkUI on application finish The Master UI now links to a reconstructed SparkUI when an application finishes, instead of keeping around a broken link. This involves exposing the path to the directory used for event logging. This commit also allows us to get rid of the SparkListenerApplicationStart event, which contains only the application name. Further, the app URL is a duplicated parameter in ApplicationInfo and ApplicationDescription, and the app name is passed as an extraneous argument to creating the backend schedulers in SparkContext. These are both fixed. TODO: Master currently does not know how to read compressed event logs. --- .../scala/org/apache/spark/SparkContext.scala | 24 +++------- .../spark/deploy/ApplicationDescription.scala | 3 +- .../apache/spark/deploy/JsonProtocol.scala | 1 - .../spark/deploy/master/ApplicationInfo.scala | 6 --- .../apache/spark/deploy/master/Master.scala | 47 +++++++++++++++++-- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/IndexPage.scala | 2 +- .../org/apache/spark/scheduler/EventBus.scala | 2 - .../scheduler/EventLoggingListener.scala | 21 ++++----- .../spark/scheduler/SparkListener.scala | 7 --- .../cluster/SparkDeploySchedulerBackend.scala | 7 ++- .../mesos/CoarseMesosSchedulerBackend.scala | 5 +- .../cluster/mesos/MesosSchedulerBackend.scala | 5 +- .../org/apache/spark/ui/JettyUtils.scala | 13 +++-- .../scala/org/apache/spark/ui/SparkUI.scala | 47 ++++++++++--------- .../org/apache/spark/ui/UIReloader.scala | 6 +-- .../org/apache/spark/ui/UISparkListener.scala | 10 ---- .../org/apache/spark/util/JsonProtocol.scala | 13 ----- .../SparkContextSchedulerCreationSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 6 +-- 20 files changed, 110 insertions(+), 119 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c58852294668e..95fa1a8ff20b1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -202,14 +202,12 @@ class SparkContext( ui.start() // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName) + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) taskScheduler.start() @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() - // Post initialization events - postApplicationStartEvent() postEnvironmentUpdateEvent() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ @@ -1039,14 +1037,6 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() - /** Post the application start event if the listener bus is ready */ - private def postApplicationStartEvent() { - Option(listenerBus).foreach { bus => - val applicationStart = SparkListenerApplicationStart(appName) - bus.post(applicationStart) - } - } - /** Post the environment update event if the listener bus is ready */ private def postEnvironmentUpdateEvent() { Option(listenerBus).foreach { bus => @@ -1223,9 +1213,7 @@ object SparkContext { } /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String, appName: String) - : TaskScheduler = - { + private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1264,7 +1252,7 @@ object SparkContext { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) scheduler @@ -1281,7 +1269,7 @@ object SparkContext { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -1337,9 +1325,9 @@ object SparkContext { val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) + new CoarseMesosSchedulerBackend(scheduler, sc, url) } else { - new MesosSchedulerBackend(scheduler, sc, url, appName) + new MesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 449b953530ff9..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -23,7 +23,8 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, val sparkHome: Option[String], - val appUiUrl: String) + var appUiUrl: String, + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index cefb1ff97e83c..c4f5e294a393e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -43,7 +43,6 @@ private[spark] object JsonProtocol { ("starttime" -> obj.startTime) ~ ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ - ("appuiurl" -> obj.appUiUrl) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ ("memoryperslave" -> obj.desc.memoryPerSlave) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index e8867bc1691d3..46b9f4dc7d3ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -31,7 +31,6 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUiUrl: String, defaultCores: Int) extends Serializable { @@ -45,11 +44,6 @@ private[spark] class ApplicationInfo( init() - private def readObject(in: java.io.ObjectInputStream) : Unit = { - in.defaultReadObject() - init() - } - private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 51794ce40cb45..7cc68925a5d2f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -37,6 +37,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -50,6 +51,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + val MAX_NUM_PERSISTED_UI = conf.getInt("spark.persisted.ui.maxConcurrent", 25) + val PERSISTED_SPARK_UI_PORT = + conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] @@ -63,6 +67,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 + val appIdToUI = new HashMap[String, SparkUI] + var nextPersistedUIPort = PERSISTED_SPARK_UI_PORT + val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling @@ -139,6 +146,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def postStop() { webUi.stop() + appIdToUI.values.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -572,8 +580,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - new ApplicationInfo( - now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores) + new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } def registerApplication(app: ApplicationInfo): Unit = { @@ -609,8 +616,31 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act }) completedApps.trimStart(toRemove) } - completedApps += app // Remember it in our history waitingApps -= app + + // If application events are logged, use them to rebuild the UI + val rebuildAppUI = app.desc.eventLogDir.isDefined + if (rebuildAppUI) { + val appName = app.desc.name + val eventLogDir = app.desc.eventLogDir.get + val ui = startPersistedSparkUI(appName, eventLogDir) + app.desc.appUiUrl = ui.appUIAddress + appIdToUI(app.id) = ui + } else { + // Avoid broken links + app.desc.appUiUrl = "" + } + completedApps += app + + // Cap the number of UIs concurrently running + if (appIdToUI.size > MAX_NUM_PERSISTED_UI) { + val oldCompletedApp = completedApps.find { oldApp => appIdToUI.contains(oldApp.id) } + oldCompletedApp.foreach { oldApp => + completedApps -= oldApp + appIdToUI.remove(oldApp.id).foreach(_.stop()) + } + } + for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id) @@ -625,6 +655,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } } + /** Start a new SparkUI rendered from persisted storage */ + def startPersistedSparkUI(appName: String, eventLogDir: String): SparkUI = { + val ui = new SparkUI(conf, nextPersistedUIPort) + ui.setAppName(appName) + ui.bind() + ui.start() + ui.renderFromPersistedStorage(eventLogDir) + nextPersistedUIPort += 1 + ui + } + /** Generate a new app ID given a app's submission date */ def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 31a431078698b..cb092cb5d576b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -82,7 +82,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index e3286606ced48..f51942803b4d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -160,7 +160,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id}
    diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index eb68b25662113..38b57d768167d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,9 +28,10 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { - private lazy val appName = parent.appName - private lazy val listener = parent.listener + private val appName = parent.appName + private val basePath = parent.basePath private val dateFmt = parent.dateFmt + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -43,7 +44,7 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Tasks

    No tasks have started yet return UIUtils.headerSparkPage( - content, appName, "Details for Stage %s".format(stageId), Stages) + content, basePath, appName, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -202,7 +203,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage(content, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 41a1942249ef2..466c014dfdf36 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -28,9 +28,10 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { + private val basePath = parent.basePath private val dateFmt = parent.dateFmt - private def isFairScheduler = parent.isFairScheduler - private def listener = parent.listener + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -75,7 +76,9 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { private def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val nameLink = - {s.name} + + {s.name} + val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) val submissionTime = s.submissionTime match { @@ -107,7 +110,8 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { {if (isFairScheduler) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index e128fff17e57c..83fcedd5ed421 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -26,13 +26,15 @@ import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { - lazy val appName = parent.appName - lazy val listener = _listener.get + val appName = parent.appName + val basePath = parent.basePath private val indexPage = new IndexPage(this) private val rddPage = new RDDPage(this) private var _listener: Option[BlockManagerListener] = None + lazy val listener = _listener.get + def start() { _listener = Some(new BlockManagerListener) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index e47f9368da62f..ae5d38f5086fc 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,14 +28,15 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private lazy val appName = parent.appName + private val appName = parent.appName + private val basePath = parent.basePath private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { // Calculate macro-level statistics val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) } /** Header fields for the RDD table */ @@ -51,7 +52,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { private def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 210f4c0257347..6f82194a108a1 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -28,7 +28,8 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private lazy val appName = parent.appName + private val appName = parent.appName + private val basePath = parent.basePath private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -91,7 +92,8 @@ private[ui] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage(content, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage( + content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) } /** Header fields for the worker table */ 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 e632630ff3202..bade1cbb70b77 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 @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc, true) + val listener = new JobProgressListener(sc.conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() diff --git a/docs/configuration.md b/docs/configuration.md index d31d97f397074..a4d3357b139c5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -140,13 +140,6 @@ Apart from these, the following properties are also available, and may be useful Port for your application's dashboard, which shows memory and workload data - - - - - From 77ba28379dce8e6cfdf1064cf286debdcb385c66 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 10 Mar 2014 18:51:48 -0700 Subject: [PATCH 55/68] Address Kay's and Patrick's comments The biggest changes include - synchronizing all methods of listeners used by the UI, refactor the StorageStatusListener such that listeners no longer extend it, and move the StorageStatusListener to storage. --- .../apache/spark/deploy/master/Master.scala | 2 +- .../spark/deploy/master/ui/MasterWebUI.scala | 9 ++- .../apache/spark/executor/TaskMetrics.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../storage/BlockManagerMasterActor.scala | 10 +-- .../storage/BlockManagerStatusListener.scala | 2 - .../StorageStatusListener.scala} | 80 +++++-------------- .../scala/org/apache/spark/ui/SparkUI.scala | 11 ++- .../org/apache/spark/ui/UIReloader.scala | 2 +- .../apache/spark/ui/env/EnvironmentUI.scala | 17 ++-- .../apache/spark/ui/exec/ExecutorsUI.scala | 17 ++-- .../spark/ui/jobs/JobProgressListener.scala | 50 ++++++------ .../org/apache/spark/ui/jobs/StagePage.scala | 4 +- .../spark/ui/storage/BlockManagerUI.scala | 51 +++++++++++- 14 files changed, 140 insertions(+), 119 deletions(-) rename core/src/main/scala/org/apache/spark/{ui/UISparkListener.scala => storage/StorageStatusListener.scala} (54%) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 0e3bd94a6f58e..8711fb3fdb33d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -653,7 +653,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act appConf.set("spark.eventLog.compress", "true") appConf.set("spark.io.compression.codec", codec) } - val ui = new SparkUI(appConf, appName, "/history/%s".format(app.id)) + val ui = new SparkUI(appConf, "%s (finished)".format(appName), "/history/%s".format(app.id)) // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = ui.renderFromPersistedStorage(eventLogDir) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 5d72f369919ad..25202cdbb3395 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -71,10 +71,11 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } def attachUI(ui: SparkUI) { - val childHandler = ui.rootHandler - rootHandler.addHandler(childHandler) - if (!childHandler.isStarted) { - childHandler.start() + for (handler <- ui.rootHandler.getHandlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 9193d247d829f..237f42b48a8c9 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -72,7 +72,7 @@ class TaskMetrics extends Serializable { var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None /** - * If blocks have been updated as a result of this task, collect the statuses of this blocks here + * Statuses of any blocks that have been updated as a result of this task. */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } 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 9b3350e66d8d2..40b836478ea5a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{Utils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index f35b002de61cf..4f77ae9ec94c0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -20,7 +20,6 @@ package org.apache.spark.storage import java.util.{HashMap => JHashMap} import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.concurrent.Future import scala.concurrent.duration._ @@ -51,7 +50,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) - private val listeners = new ArrayBuffer[BlockManagerStatusListener] + // Post block manager status updates to SparkContext through a listener + private var statusListener: Option[BlockManagerStatusListener] = None val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong @@ -71,7 +71,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } def registerListener(listener: BlockManagerStatusListener) { - listeners += listener + statusListener = Some(listener) } def receive = { @@ -168,7 +168,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } } val blockManagerLost = SparkListenerBlockManagerLost(blockManagerId) - listeners.foreach(_.onBlockManagerLost(blockManagerLost)) + statusListener.foreach(_.onBlockManagerLost(blockManagerLost)) } private def expireDeadHosts() { @@ -246,7 +246,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act maxMemSize, slaveActor) } val blockManagerGained = SparkListenerBlockManagerGained(id, maxMemSize) - listeners.foreach(_.onBlockManagerGained(blockManagerGained)) + statusListener.foreach(_.onBlockManagerGained(blockManagerGained)) } private def updateBlockInfo( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala index 0a88ea79b2774..adc356ec8cf4e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala @@ -35,12 +35,10 @@ import org.apache.spark.scheduler._ * event should be buffered. */ private[spark] class BlockManagerStatusListener extends SparkListener { - private var _listenerBus: Option[SparkListenerBus] = None // Buffer any events received before the listener bus is ready private val bufferedEvents = new ArrayBuffer[SparkListenerEvent] - with mutable.SynchronizedBuffer[SparkListenerEvent] /** * Set the listener bus. If there are buffered events, post them all to the listener bus. diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala similarity index 54% rename from core/src/main/scala/org/apache/spark/ui/UISparkListener.scala rename to core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 405144bbbf9f0..2aaf02869b18a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -15,20 +15,17 @@ * limitations under the License. */ -package org.apache.spark.ui +package org.apache.spark.storage import scala.collection.mutable import org.apache.spark.scheduler._ -import org.apache.spark.storage._ - -private[ui] trait UISparkListener extends SparkListener /** * A SparkListener that maintains executor storage status */ -private[ui] class StorageStatusSparkListener extends UISparkListener { - val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() +private[spark] class StorageStatusListener extends SparkListener { + private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() def storageStatusList = executorIdToStorageStatus.values.toSeq @@ -52,7 +49,7 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo if (info != null) { val execId = formatExecutorId(info.executorId) @@ -66,71 +63,34 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { } } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { updateStorageStatus(unpersistRDD.rddId) } override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { - val blockManagerId = blockManagerGained.blockManagerId - val executorId = blockManagerId.executorId - val maxMem = blockManagerGained.maxMem - val storageStatus = new StorageStatus(blockManagerId, maxMem) - executorIdToStorageStatus(executorId) = storageStatus + synchronized { + val blockManagerId = blockManagerGained.blockManagerId + val executorId = blockManagerId.executorId + val maxMem = blockManagerGained.maxMem + val storageStatus = new StorageStatus(blockManagerId, maxMem) + executorIdToStorageStatus(executorId) = storageStatus + } } override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) { - val executorId = blockManagerLost.blockManagerId.executorId - executorIdToStorageStatus.remove(executorId) + synchronized { + val executorId = blockManagerLost.blockManagerId.executorId + executorIdToStorageStatus.remove(executorId) + } } /** * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). This results in - * duplicate rows for the same executor. Thus, in this mode, we aggregate these two - * rows and use the executor ID of "" to be consistent. + * task ("localhost") and that according to SparkEnv (""). In the UI, this + * results in duplicate rows for the same executor. Thus, in this mode, we aggregate + * these two rows and use the executor ID of "" to be consistent. */ - protected def formatExecutorId(execId: String): String = { + def formatExecutorId(execId: String): String = { if (execId == "localhost") "" else execId } - -} - -/** - * A SparkListener that maintains RDD information - */ -private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { - private val _rddInfoMap = mutable.Map[Int, RDDInfo]() - - /** Filter RDD info to include only those with cached partitions */ - def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - - /** Update each RDD's info to reflect any updates to the RDD's storage status */ - private def updateRDDInfo() { - val updatedRDDInfoList = StorageUtils.rddInfoFromStorageStatus(storageStatusList, _rddInfoMap) - updatedRDDInfoList.foreach { info => _rddInfoMap(info.id) = info } - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - super.onTaskEnd(taskEnd) - val metrics = taskEnd.taskMetrics - if (metrics != null && metrics.updatedBlocks.isDefined) { - updateRDDInfo() - } - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap(rddInfo.id) = rddInfo - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - // Remove all partitions that are no longer cached - _rddInfoMap.retain { case (id, info) => info.numCachedPartitions > 0 } - } - - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { - super.onUnpersistRDD(unpersistRDD) - updateRDDInfo() - } - } 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 1b519465ca3b4..68cfd9edc6f88 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -22,6 +22,7 @@ import org.eclipse.jetty.server.handler.ContextHandlerCollection import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.scheduler.{EventLoggingListener, EventLoggingInfo, SparkReplayerBus} +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI @@ -83,6 +84,9 @@ private[spark] class SparkUI( collection } + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener + // Only log events if this SparkUI is live private var eventLogger: Option[EventLoggingListener] = None @@ -109,10 +113,6 @@ private[spark] class SparkUI( /** Initialize all components of the server */ def start() { - // NOTE: This is decoupled from bind() because of the following dependency cycle: - // DAGScheduler() requires that the port of this server is known - // This server must register all handlers, including JobProgressUI, before binding - // JobProgressUI registers a listener with SparkContext, which requires sc to initialize storage.start() jobs.start() env.start() @@ -131,6 +131,9 @@ private[spark] class SparkUI( replayerBus = Some(new SparkReplayerBus(conf)) replayerBus.get } + + // Storage status listener must receive events first, as other listeners depend on its state + eventBus.addListener(storageStatusListener) eventBus.addListener(storage.listener) eventBus.addListener(jobs.listener) eventBus.addListener(env.listener) diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala index d2863a4033e1c..355552f1b2326 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import org.apache.spark.SparkConf /** - * A simple example that reloads a persisted UI independently from a SparkContext + * Reload a persisted UI independently from a SparkContext. */ object UIReloader { def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index cddf9a3e23273..d498c882fe852 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -20,12 +20,13 @@ package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest import scala.xml.Node + import org.eclipse.jetty.server.Handler import org.apache.spark.scheduler._ +import org.apache.spark.ui._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment -import org.apache.spark.ui._ private[ui] class EnvironmentUI(parent: SparkUI) { private val appName = parent.appName @@ -72,17 +73,19 @@ private[ui] class EnvironmentUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the EnvironmentUI */ -private[ui] class EnvironmentListener extends UISparkListener { +private[ui] class EnvironmentListener extends SparkListener { var jvmInformation: Seq[(String, String)] = Seq() var sparkProperties: Seq[(String, String)] = Seq() var systemProperties: Seq[(String, String)] = Seq() var classpathEntries: Seq[(String, String)] = Seq() override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - val environmentDetails = environmentUpdate.environmentDetails - jvmInformation = environmentDetails("JVM Information") - sparkProperties = environmentDetails("Spark Properties") - systemProperties = environmentDetails("System Properties") - classpathEntries = environmentDetails("Classpath Entries") + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 86c02538cd399..af20b307a1b39 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -26,6 +26,7 @@ import org.eclipse.jetty.server.Handler import org.apache.spark.ExceptionFailure import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui._ @@ -39,7 +40,7 @@ private[ui] class ExecutorsUI(parent: SparkUI) { lazy val listener = _listener.get def start() { - _listener = Some(new ExecutorsListener()) + _listener = Some(new ExecutorsListener(parent.storageStatusListener)) } def getHandlers = Seq[(String, Handler)]( @@ -160,7 +161,9 @@ private[ui] class ExecutorsUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the ExecutorsUI */ -private[ui] class ExecutorsListener extends StorageStatusSparkListener { +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() @@ -168,12 +171,14 @@ private[ui] class ExecutorsListener extends StorageStatusSparkListener { val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() - override def onTaskStart(taskStart: SparkListenerTaskStart) { + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val eid = formatExecutorId(taskStart.taskInfo.executorId) executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = formatExecutorId(info.executorId) @@ -198,7 +203,9 @@ private[ui] class ExecutorsListener extends StorageStatusSparkListener { executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten } } - super.onTaskEnd(taskEnd) } } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index f80576e0bcf48..5b1816f41e2bc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -21,19 +21,18 @@ import scala.collection.mutable.{ListBuffer, HashMap} import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId -import org.apache.spark.ui.UISparkListener /** * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the - * class, since the UI thread and the DAGScheduler event loop may otherwise - * be reading/updating the internal data structures concurrently. + * class, since the UI thread and the EventBus loop may otherwise be reading and + * updating the internal data structures concurrently. */ -private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { +private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { import JobProgressListener._ @@ -57,7 +56,7 @@ private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { val stageIdToTasksActive = HashMap[Int, HashMap[Long, TaskInfo]]() val stageIdToTasksComplete = HashMap[Int, Int]() val stageIdToTasksFailed = HashMap[Int, Int]() - val stageIdToTaskInfos = HashMap[Int, HashMap[Long, TaskUIData]]() + val stageIdToTaskData = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() val stageIdToPool = HashMap[Int, String]() val stageIdToDescription = HashMap[Int, String]() @@ -84,7 +83,7 @@ private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { if (stages.size > retainedStages) { val toRemove = retainedStages / 10 stages.takeRight(toRemove).foreach( s => { - stageIdToTaskInfos.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -125,14 +124,13 @@ private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { if (taskInfo != null) { val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) tasksActive(taskInfo.taskId) = taskInfo - val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) - stageIdToTaskInfos(sid) = taskMap + stageIdToTaskData(sid) = taskMap } } - override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) - = synchronized { + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { // Do nothing: because we don't do a deep copy of the TaskInfo, the TaskInfo in // stageToTaskInfos already has the updated status. } @@ -211,9 +209,9 @@ private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) stageIdToDiskBytesSpilled(sid) += diskBytesSpilled - val taskMap = stageIdToTaskInfos.getOrElse(sid, HashMap[Long, TaskUIData]()) + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) taskMap(info.taskId) = new TaskUIData(info, metrics, failureInfo) - stageIdToTaskInfos(sid) = taskMap + stageIdToTaskData(sid) = taskMap } } @@ -232,23 +230,29 @@ private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { } override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None + synchronized { + val schedulingModeName = + environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") + schedulingMode = schedulingModeName match { + case Some(name) => Some(SchedulingMode.withName(name)) + case None => None + } } } override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { - val blockManagerId = blockManagerGained.blockManagerId - val executorId = blockManagerId.executorId - executorIdToBlockManagerId(executorId) = blockManagerId + synchronized { + val blockManagerId = blockManagerGained.blockManagerId + val executorId = blockManagerId.executorId + executorIdToBlockManagerId(executorId) = blockManagerId + } } override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) { - val executorId = blockManagerLost.blockManagerId.executorId - executorIdToBlockManagerId.remove(executorId) + synchronized { + val executorId = blockManagerLost.blockManagerId.executorId + executorIdToBlockManagerId.remove(executorId) + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 38b57d768167d..1427b1f6e0458 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -37,7 +37,7 @@ private[ui] class StagePage(parent: JobProgressUI) { listener.synchronized { val stageId = request.getParameter("id").toInt - if (!listener.stageIdToTaskInfos.contains(stageId)) { + if (!listener.stageIdToTaskData.contains(stageId)) { val content =

    Summary Metrics

    No tasks have started yet @@ -47,7 +47,7 @@ private[ui] class StagePage(parent: JobProgressUI) { content, basePath, appName, "Details for Stage %s".format(stageId), Stages) } - val tasks = listener.stageIdToTaskInfos(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) + val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 83fcedd5ed421..e0ba1b8da7d3d 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -19,10 +19,14 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.collection.mutable + import org.eclipse.jetty.server.Handler -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui._ +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { @@ -36,7 +40,7 @@ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val listener = _listener.get def start() { - _listener = Some(new BlockManagerListener) + _listener = Some(new BlockManagerListener(parent.storageStatusListener)) } def getHandlers = Seq[(String, Handler)]( @@ -48,4 +52,45 @@ private[ui] class BlockManagerUI(parent: SparkUI) { /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[ui] class BlockManagerListener extends RDDInfoSparkListener +private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + + def storageStatusList = storageStatusListener.storageStatusList + + /** Filter RDD info to include only those with cached partitions */ + def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + + /** Update each RDD's info to reflect any updates to the RDD's storage status */ + private def updateRDDInfo() { + val updatedRDDInfoList = StorageUtils.rddInfoFromStorageStatus(storageStatusList, _rddInfoMap) + updatedRDDInfoList.foreach { info => _rddInfoMap(info.id) = info } + } + + /** + * Assumes the storage status list is fully up-to-date. This implies that the corresponding + * StorageStatusSparkListener must + */ + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val metrics = taskEnd.taskMetrics + if (metrics != null && metrics.updatedBlocks.isDefined) { + updateRDDInfo() + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + val rddInfo = stageSubmitted.stageInfo.rddInfo + _rddInfoMap(rddInfo.id) = rddInfo + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + // Remove all partitions that are no longer cached + _rddInfoMap.retain { case (id, info) => info.numCachedPartitions > 0 } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + super.onUnpersistRDD(unpersistRDD) + updateRDDInfo() + } +} From dc93915ae051ffc2d855af73b5f7f174f34d56a1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 10 Mar 2014 22:06:41 -0700 Subject: [PATCH 56/68] Imports, comments, and code formatting (minor) --- .../scala/org/apache/spark/SparkEnv.scala | 6 ++--- .../apache/spark/deploy/master/Master.scala | 4 +-- .../spark/deploy/master/ui/MasterWebUI.scala | 5 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 14 +++++----- .../org/apache/spark/scheduler/EventBus.scala | 2 +- .../apache/spark/scheduler/JobLogger.scala | 9 +++++-- .../spark/scheduler/SparkListener.scala | 13 +++++++--- .../spark/scheduler/SparkReplayerBus.scala | 6 ++--- .../apache/spark/scheduler/StageInfo.scala | 3 +-- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 12 ++++----- .../storage/BlockManagerMasterActor.scala | 4 +-- .../storage/BlockManagerStatusListener.scala | 1 - .../apache/spark/storage/MemoryStore.scala | 5 ++-- .../apache/spark/storage/StorageUtils.scala | 5 ++-- .../scala/org/apache/spark/ui/SparkUI.scala | 9 ++++--- .../org/apache/spark/ui/UIReloader.scala | 4 +-- .../scala/org/apache/spark/ui/UIUtils.scala | 5 ++-- .../apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 12 ++++----- .../apache/spark/ui/jobs/ExecutorTable.scala | 26 +++++++++---------- .../org/apache/spark/ui/jobs/IndexPage.scala | 9 ++++--- .../spark/ui/jobs/JobProgressListener.scala | 17 ++++++------ .../org/apache/spark/ui/jobs/PoolPage.scala | 12 +++++---- .../org/apache/spark/ui/jobs/PoolTable.scala | 6 ++--- .../apache/spark/ui/storage/IndexPage.scala | 1 - .../org/apache/spark/util/FileLogger.scala | 6 ++--- .../org/apache/spark/util/JsonProtocol.scala | 14 +++++----- .../scala/org/apache/spark/util/Utils.scala | 5 ++-- .../org/apache/spark/CacheManagerSuite.scala | 4 +-- .../apache/spark/JobCancellationSuite.scala | 2 +- .../SparkContextSchedulerCreationSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 5 +--- .../spark/scheduler/DAGSchedulerSuite.scala | 8 ++++-- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- 35 files changed, 124 insertions(+), 118 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d4229a22a7b03..34831f21e5f15 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -33,7 +33,6 @@ import org.apache.spark.serializer.{Serializer, SerializerManager} import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} - /** * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently @@ -251,7 +250,7 @@ object SparkEnv extends Logging { /** * Return a map representation of jvm information, Spark properties, system properties, and * class paths. Map keys define the category, and map values represent the corresponding - * attributes as a sequence of KV pairs. + * attributes as a sequence of KV pairs. This is used mainly for SparkListenerEnvironmentUpdate. */ private[spark] def environmentDetails( @@ -274,12 +273,11 @@ object SparkEnv extends Logging { } val sparkProperties = conf.getAll.sorted ++ additionalFields + // System properties that are not java classpaths val systemProperties = System.getProperties.iterator.toSeq val classPathProperty = systemProperties.find { case (k, v) => k == "java.class.path" }.getOrElse(("", "")) - - // System properties that are not java classpaths val otherProperties = systemProperties.filter { case (k, v) => k != "java.class.path" && !k.startsWith("spark.") }.sorted diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8711fb3fdb33d..4bdfa9f966e05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -618,8 +618,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // If application events are logged, use them to rebuild the UI startPersistedSparkUI(app).map { ui => app.desc.appUiUrl = ui.basePath - webUi.attachUI(ui) appIdToUI(app.id) = ui + webUi.attachUI(ui) }.getOrElse { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" @@ -640,7 +640,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } /** - * Start a new SparkUI rendered from persisted storage. If unsuccessful for any reason, + * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, * return None. Otherwise return the reconstructed UI. */ def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 25202cdbb3395..96c6fa060f27a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -32,14 +32,13 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - private val host = Utils.localHostName() - private val port = requestedPort - val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) var server: Option[Server] = None var boundPort: Option[Int] = None + private val host = Utils.localHostName() + private val port = requestedPort private val applicationPage = new ApplicationPage(this) private val indexPage = new IndexPage(this) 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 40b836478ea5a..b152912e0a044 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -553,11 +553,11 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach { handleJobCancellation } + jobIds.foreach(handleJobCancellation) case AllJobsCancelled => // Cancel all running jobs. - runningStages.map(_.jobId).foreach { handleJobCancellation } + runningStages.map(_.jobId).foreach(handleJobCancellation) activeJobs.clear() // These should already be empty by this point, stageIdToActiveJob.clear() // but just in case we lost track of some jobs... @@ -1094,11 +1094,11 @@ class DAGScheduler( "stageToInfos" -> stageToInfos, "jobIdToStageIds" -> jobIdToStageIds, "stageIdToJobIds" -> stageIdToJobIds). - foreach { case(s, t) => { - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - }} + foreach { case(s, t) => + val sizeBefore = t.size + t.clearOldValues(cleanupTime) + logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) + } } def stop() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala index a98ec5f05710c..2176fbc11fc55 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer /** - * A SparkListenerEvent bus that relays events to its listeners. + * A SparkListenerEvent bus that relays events to its listeners */ private[spark] trait EventBus { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 6c257276bdd4e..0778e4820e9b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -32,10 +32,15 @@ import org.apache.spark.executor.TaskMetrics * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext * is created. Note that each JobLogger only works for one SparkContext + * + * NOTE: The functionality of this class is heavily stripped down to accommodate for a general + * refactor of the SparkListener interface. In its place, the EventLoggingListener is introduced + * to log application information as SparkListenerEvents through the SparkUI. To enable this + * functionality, set spark.eventLog.enabled to true. */ -class JobLogger(val user: String, val logDirName: String) - extends SparkListener with Logging { +@deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") +class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) 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 36873bfa90ba8..02a68146a8b0c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -21,10 +21,10 @@ import java.util.Properties import scala.collection.Map -import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.storage.{BlockManagerId, StorageStatus} +import org.apache.spark.storage.BlockManagerId sealed trait SparkListenerEvent @@ -37,8 +37,13 @@ case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends Spar case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerTaskEnd(stageId: Int, taskType: String, reason: TaskEndReason, - taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvent +case class SparkListenerTaskEnd( + stageId: Int, + taskType: String, + reason: TaskEndReason, + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) + extends SparkListenerEvent case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala index f17a87ec2de8c..63db674cb0599 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala @@ -25,12 +25,12 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{SparkConf, Logging} -import org.apache.spark.util.{Utils, JsonProtocol} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} /** - * An EventBus that replays logged events from persisted storage. + * An EventBus that replays logged events from persisted storage */ private[spark] class SparkReplayerBus(conf: SparkConf) extends EventBus with Logging { private val compressed = conf.getBoolean("spark.eventLog.compress", false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 577932474c1fc..3dfc1af2892ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -32,8 +32,7 @@ class StageInfo( val name: String, val numTasks: Int, val rddInfo: RDDInfo, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = - mutable.Buffer[(TaskInfo, TaskMetrics)]()) { + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer.empty) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 395a160cc3f37..b1bee7beebe7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c35cc6abbe35e..e8ea9e1ceaa48 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.collection.mutable.{HashMap, ArrayBuffer} +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random @@ -122,11 +122,11 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this( - execId: String, - actorSystem: ActorSystem, - master: BlockManagerMaster, - serializer: Serializer, - conf: SparkConf) = { + execId: String, + actorSystem: ActorSystem, + master: BlockManagerMaster, + serializer: Serializer, + conf: SparkConf) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 4f77ae9ec94c0..1eac0b6863d9c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -242,8 +242,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act case None => blockManagerIdByExecutor(id.executorId) = id } - blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), - maxMemSize, slaveActor) + blockManagerInfo(id) = + new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } val blockManagerGained = SparkListenerBlockManagerGained(id, maxMemSize) statusListener.foreach(_.onBlockManagerGained(blockManagerGained)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala index adc356ec8cf4e..9af73d5d88779 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala @@ -17,7 +17,6 @@ package org.apache.spark.storage -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 11f1040ae505f..2d3ceb2b89a05 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -284,5 +284,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } -private case class ResultWithDroppedBlocks(success: Boolean, - droppedBlocks: Seq[(BlockId, BlockStatus)]) +private case class ResultWithDroppedBlocks( + success: Boolean, + droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 07e0ec1ffbced..a1d1393348d49 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,13 +17,12 @@ package org.apache.spark.storage -import scala.collection.mutable import scala.collection.Map +import scala.collection.mutable import org.apache.spark.SparkContext import org.apache.spark.util.Utils - private[spark] class StorageStatus( val blockManagerId: BlockManagerId, @@ -74,7 +73,7 @@ object StorageUtils { /** Returns RDD-level information from a list of StorageStatus objects and SparkContext */ def rddInfoFromStorageStatus( storageStatusList: Seq[StorageStatus], - sc: SparkContext) : Array[RDDInfo] = { + sc: SparkContext): Array[RDDInfo] = { val blockStatusMap = blockStatusMapFromStorageStatus(storageStatusList) val rddInfoList = rddInfoFromSparkContext(blockStatusMap.keys.toSeq, sc) val rddInfoMap = rddInfoList.map { info => (info.id, info) }.toMap 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 68cfd9edc6f88..1a957bcd41b75 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,7 +21,7 @@ import org.eclipse.jetty.server.{Handler, Server} import org.eclipse.jetty.server.handler.ContextHandlerCollection import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} -import org.apache.spark.scheduler.{EventLoggingListener, EventLoggingInfo, SparkReplayerBus} +import org.apache.spark.scheduler.{EventLoggingInfo, EventLoggingListener, SparkReplayerBus} import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI @@ -38,6 +38,8 @@ private[spark] class SparkUI( val basePath: String = "") extends Logging { + import SparkUI._ + def this(sc: SparkContext) = this(sc, sc.conf, sc.appName) def this(conf: SparkConf, appName: String) = this(null, conf, appName) def this(conf: SparkConf, appName: String, basePath: String) = @@ -47,7 +49,7 @@ private[spark] class SparkUI( val live = sc != null private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + private val port = conf.get("spark.ui.port", DEFAULT_PORT).toInt private var boundPort: Option[Int] = None private var server: Option[Server] = None private var started = false @@ -69,7 +71,7 @@ private[spark] class SparkUI( exec.getHandlers ++ metricsServletHandlers ++ Seq[(String, Handler)]( - ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), + ("/static", createStaticHandler(STATIC_RESOURCE_DIR)), ("/", createRedirectHandler("/stages", basePath)) ) } @@ -165,6 +167,5 @@ private[spark] class SparkUI( private[spark] object SparkUI { val DEFAULT_PORT = "4040" - val DEFAULT_PERSISTED_PORT = "14040" val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala index 355552f1b2326..f58e36a213b98 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -25,13 +25,13 @@ import org.apache.spark.SparkConf object UIReloader { def main(args: Array[String]) { if (args.length < 1) { - println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path] [port]") + println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path]") System.exit(1) } val conf = new SparkConf() conf.set("spark.ui.port", "14040") - val ui = new SparkUI(conf, "Reloaded Application") + val ui = new SparkUI(conf, "My Application") ui.bind() ui.start() val success = ui.renderFromPersistedStorage(args(0)) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index aba2088d5c6b3..beb1d1ce386c9 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -21,6 +21,7 @@ import scala.xml.Node /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { + import Page._ // Yarn has to go through a proxy so the base uri is provided and has to be on all links @@ -62,8 +63,8 @@ private[spark] object UIUtils { - + {appName} - {title} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index d498c882fe852..ef0ecd14097e6 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -36,7 +36,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) { lazy val listener = _listener.get def start() { - _listener = Some(new EnvironmentListener()) + _listener = Some(new EnvironmentListener) } def getHandlers = Seq[(String, Handler)]( diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index af20b307a1b39..a8076cc4a2396 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -29,7 +29,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui._ +import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { @@ -67,11 +67,11 @@ private[ui] class ExecutorsUI(parent: SparkUI) {
    -
    -
    - {execTable} -
    -
    ; +
    +
    + {execTable} +
    +
    ; UIUtils.headerSparkPage( content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 0ae32c9b56283..73861ae6746da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -66,20 +66,20 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { executorIdToSummary match { case Some(x) => x.toSeq.sortBy(_._1).map { case (k, v) => { -
    - - - - - - - - - - - - } + + + + + + + + + + + + } + } case _ => Seq[Node]() } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 5846090d4ee6b..f3c93d4214ad0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -19,8 +19,9 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{Node, NodeSeq} +import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils @@ -45,8 +46,8 @@ private[ui] class IndexPage(parent: JobProgressUI) { new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - // For now, pool information is only accessible in live UI's - val pools = if (live) sc.getAllPools else Seq() + // For now, pool information is only accessible in live UIs + val pools = if (live) sc.getAllPools else Seq[Schedulable]() val poolTable = new PoolTable(pools, parent) val summary: NodeSeq = @@ -82,7 +83,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { {if (live && isFairScheduler) {

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq } else { - Seq() + Seq[Node]() }} ++

    Active Stages ({activeStages.size})

    ++ activeStagesTable.toNodeSeq ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5b1816f41e2bc..7cd4183d51889 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{ListBuffer, HashMap} +import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} import org.apache.spark.executor.TaskMetrics @@ -185,28 +185,27 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(m => m.executorRunTime).getOrElse(0L) + val time = metrics.map(_.executorRunTime).getOrElse(0L) stageIdToTime(sid) += time totalTime += time stageIdToShuffleRead.getOrElseUpdate(sid, 0L) - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) + val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) stageIdToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) + val shuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) stageIdToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) - val memoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled).getOrElse(0L) + val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) - val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) + val diskBytesSpilled = metrics.map(_.diskBytesSpilled).getOrElse(0L) stageIdToDiskBytesSpilled(sid) += diskBytesSpilled val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) @@ -263,6 +262,6 @@ private[ui] case class TaskUIData( exception: Option[ExceptionFailure] = None) private object JobProgressListener { - val DEFAULT_RETAINED_STAGES = 1000 val DEFAULT_POOL_NAME = "default" + val DEFAULT_RETAINED_STAGES = 1000 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index eddb024d07b7b..bd33182b70059 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,6 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils @@ -38,16 +39,17 @@ private[ui] class PoolPage(parent: JobProgressUI) { val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.get(poolName) match { case Some(s) => s.values.toSeq - case None => Seq() + case None => Seq[StageInfo]() } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - // For now, pool information is only accessible in live UI's - val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq() + // For now, pool information is only accessible in live UIs + val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() val poolTable = new PoolTable(pools, parent) - val content =

    Summary

    ++ poolTable.toNodeSeq ++ -

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq + val content = +

    Summary

    ++ poolTable.toNodeSeq ++ +

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq UIUtils.headerSparkPage( content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 6dbd8a54615a5..f565fa96d273b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -37,8 +37,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { private def poolTable( makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], - rows: Seq[Schedulable]) - : Seq[Node] = { + rows: Seq[Schedulable]) : Seq[Node] = {
    {d}
    - + {rdd.name} spark.logConf false - Log the supplied SparkConf as INFO at start of spark context. + Whether to log the supplied SparkConf as INFO at start of spark context. +
    spark.eventLog.enabledfalse + Whether to log spark events, useful for reconstructing the Web UI after the application has finished. +
    spark.eventLog.dir/tmp/spark-events + Directory in which spark events are logged, if spark.eventLog.enabled is true.
    spark.persisted.ui.port14040 + Port on which a Spark UI rebuilt from persisted storage is hosted. +
    spark.ui.retainedStages 1000
    spark.eventLog.compressfalse + Whether to compress logged events, if spark.eventLog.enabled is true. +
    spark.eventLog.dir /tmp/spark-events - Directory in which spark events are logged, if spark.eventLog.enabled is true. + Base directory in which spark events are logged, if spark.eventLog.enabled is true. + Within this base directory, Spark creates a sub-directory for each application, and logs the events + specific to the application in this directory.
    - {app.desc.name} + {app.desc.name} {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala index 529ee1a8d09ca..f2c1863a6e9e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala @@ -62,8 +62,6 @@ private[spark] trait EventBus { listeners.foreach(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => listeners.foreach(_.onTaskEnd(taskEnd)) - case applicationStart: SparkListenerApplicationStart => - listeners.foreach(_.onApplicationStart(applicationStart)) case environmentUpdate: SparkListenerEnvironmentUpdate => listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) case executorsStateChange: SparkListenerExecutorsStateChange => 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 52a7c9f6dfc97..3a92c1a2c6d2b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -35,28 +35,24 @@ import org.apache.spark.{Logging, SparkConf} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { - private val shouldLog = conf.getBoolean("spark.eventLog.enabled", false) private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", true) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") private val name = appName.replaceAll("[ /]", "-").toLowerCase + "-" + System.currentTimeMillis() - private val logDir = logBaseDir + "/" + name + val logDir = logBaseDir + "/" + name - private val logger: Option[FileLogger] = if (shouldLog) { - logInfo("Logging events to %s".format(logDir)) - Some(new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite)) - } else { - logWarning("Event logging is disabled. To enable it, set spark.eventLog.enabled to true.") - None - } + private val logger = + new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + + logInfo("Logging events to %s".format(logDir)) /** Log the event as JSON */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) - logger.foreach(_.logLine(eventJson)) + logger.logLine(eventJson) if (flushLogger) { - logger.foreach(_.flush()) + logger.flush() } } @@ -65,7 +61,6 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) override def onTaskStart(event: SparkListenerTaskStart) = logEvent(event) override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = logEvent(event) override def onTaskEnd(event: SparkListenerTaskEnd) = logEvent(event) - override def onApplicationStart(event: SparkListenerApplicationStart) = logEvent(event) override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = logEvent(event) // Events that trigger a flush @@ -80,5 +75,5 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) - def stop() = logger.foreach(_.stop()) + def stop() = logger.stop() } 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 56d7a012f5068..8674af928e370 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -45,8 +45,6 @@ case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Pro case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -case class SparkListenerApplicationStart(appName: String) extends SparkListenerEvent - case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent @@ -99,11 +97,6 @@ trait SparkListener { */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } - /** - * Called when the application starts - */ - def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } - /** * Called when environment properties have been updated */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ee4b65e312abc..f490d35ac6b78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -26,8 +26,7 @@ import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - masters: Array[String], - appName: String) + masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with AppClientListener with Logging { @@ -49,8 +48,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, - sparkHome, "http://" + sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, + sparkHome, sc.ui.appUIAddress, sc.ui.eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 28b019d9fd495..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -45,8 +45,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend private[spark] class CoarseMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with MScheduler with Logging { @@ -94,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c576beb0c0d38..32ec2b56da0b5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -41,8 +41,7 @@ import org.apache.spark.util.Utils private[spark] class MesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends SchedulerBackend with MScheduler with Logging { @@ -71,7 +70,7 @@ private[spark] class MesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 1b78c52ff6077..8f990b55c585e 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -49,8 +49,11 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, - extractFn: T => String = (in: Any) => in.toString): Handler = { + def createHandler[T <% AnyRef]( + responder: Responder[T], + contentType: String, + extractFn: T => String = (in: Any) => in.toString): Handler = { + new AbstractHandler { def handle(target: String, baseRequest: Request, @@ -99,8 +102,10 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) - = { + def startJettyServer( + hostName: String, + port: Int, + handlers: Seq[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => val contextHandler = new ContextHandler(path) 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 5f5dd24c4f2a3..b8a5d2dfc3332 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -29,24 +29,25 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark. */ -private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logging { +private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf, port: Int) extends Logging { - def this() = this(null, new SparkConf()) - def this(conf: SparkConf) = this(null, conf) - def this(sc: SparkContext) = this(sc, sc.conf) + def this(sc: SparkContext) = + this(sc, sc.conf, sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt) + + // Persisted UI constructors + def this(conf: SparkConf, port: Int) = this(null, conf, port) + def this(conf: SparkConf) = + this(conf, conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt) + def this() = this(new SparkConf()) // If SparkContext is not provided, assume this UI is rendered from persisted storage val live = sc != null - val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - var port = if (live) { - conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - } else { - conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt - } - var boundPort: Option[Int] = None - var server: Option[Server] = None - var started = false - var appName = "" + var appName = if (live) sc.appName else "" + + private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private var boundPort: Option[Int] = None + private var server: Option[Server] = None + private var started = false private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -68,10 +69,6 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers - - // A simple listener that sets the app name for this SparkUI - private val appNameListener = new AppNameListener(this) - // Only log events if this SparkUI is live private var eventLogger: Option[EventLoggingListener] = None @@ -80,6 +77,9 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg def setAppName(name: String) = appName = name + // Path to directory in which events are logged, if any + def eventLogDir: Option[String] = eventLogger.map { l => Some(l.logDir) }.getOrElse(None) + /** Bind the HTTP server which backs this web interface */ def bind() { try { @@ -107,8 +107,12 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg // Listen for events from the SparkContext if it exists, otherwise from persisted storage val eventBus = if (live) { - eventLogger = Some(new EventLoggingListener(sc.appName, conf)) - sc.listenerBus.addListener(eventLogger.get) + val loggingEnabled = conf.getBoolean("spark.eventLog.enabled", false) + if (loggingEnabled) { + val logger = new EventLoggingListener(appName, conf) + eventLogger = Some(logger) + sc.listenerBus.addListener(logger) + } sc.listenerBus } else { replayerBus = Some(new SparkReplayerBus(conf)) @@ -118,7 +122,6 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg eventBus.addListener(jobs.listener) eventBus.addListener(env.listener) eventBus.addListener(exec.listener) - eventBus.addListener(appNameListener) started = true } @@ -140,7 +143,7 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf) extends Logg logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") + private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala index 01de077782fc9..c77fcff561ad9 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -29,10 +29,10 @@ object UIReloader { System.exit(1) } - val port = if (args.length == 2) args(1) else "14040" + val port = if (args.length == 2) args(1).toInt else 14040 val conf = new SparkConf() - conf.set("spark.persisted.ui.port", port) - val ui = new SparkUI(conf) + val ui = new SparkUI(conf, port) + ui.setAppName("Reloaded Application") ui.bind() ui.start() val success = ui.renderFromPersistedStorage(args(0)) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index c696f0275a9c2..61ecf0e5e882d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -24,16 +24,6 @@ import org.apache.spark.storage._ private[ui] trait UISparkListener extends SparkListener -/** - * A SparkListener that listens only for application start events to set the app name for the UI. - */ -private[ui] class AppNameListener(parent: SparkUI) extends UISparkListener { - override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - val appName = applicationStart.appName - parent.setAppName(appName) - } -} - /** * A SparkListener that maintains executor storage status */ 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 07fb80e5e6a9a..3764577b7035f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -55,8 +55,6 @@ private[spark] object JsonProtocol { jobStartToJson(jobStart) case jobEnd: SparkListenerJobEnd => jobEndToJson(jobEnd) - case applicationStart: SparkListenerApplicationStart => - applicationStartToJson(applicationStart) case environmentUpdate: SparkListenerEnvironmentUpdate => environmentUpdateToJson(environmentUpdate) case executorsStateChange: SparkListenerExecutorsStateChange => @@ -126,11 +124,6 @@ private[spark] object JsonProtocol { ("Job Result" -> jobResult) } - def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { - ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ - ("App Name" -> applicationStart.appName) - } - def environmentUpdateToJson(environmentUpdate: SparkListenerEnvironmentUpdate): JValue = { val environmentDetails = environmentUpdate.environmentDetails val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) @@ -389,7 +382,6 @@ private[spark] object JsonProtocol { val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) - val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) val executorsStateChanged = Utils.getFormattedClassName(SparkListenerExecutorsStateChange) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) @@ -403,7 +395,6 @@ private[spark] object JsonProtocol { case `taskEnd` => taskEndFromJson(json) case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) - case `applicationStart` => applicationStartFromJson(json) case `environmentUpdate` => environmentUpdateFromJson(json) case `executorsStateChanged` => executorsStateChangeFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) @@ -455,10 +446,6 @@ private[spark] object JsonProtocol { SparkListenerJobEnd(jobId, jobResult) } - def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { - SparkListenerApplicationStart((json \ "App Name").extract[String]) - } - def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { val environmentDetails = Map[String, Seq[(String, String)]]( "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f28d5c7b133b3..0e44e41813d77 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -32,7 +32,7 @@ class SparkContextSchedulerCreationSuite // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test") + val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index bae3b37e267d5..0c23495819111 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -96,7 +96,7 @@ class JsonProtocolSuite extends FunSuite { def createAppInfo() : ApplicationInfo = { val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime, - "id", createAppDesc(), JsonConstants.submitDate, null, "appUriStr", Int.MaxValue) + "id", createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) appInfo.endTime = JsonConstants.currTimeInMillis appInfo } @@ -148,12 +148,12 @@ object JsonConstants { val submitDate = new Date(123456789) val appInfoJsonStr = """ - |{"starttime":3,"id":"id","name":"name","appuiurl":"appUriStr", + |{"starttime":3,"id":"id","name":"name", |"cores":4,"user":"%s", |"memoryperslave":1234,"submitdate":"%s", |"state":"WAITING","duration":%d} """.format(System.getProperty("user.name", ""), - submitDate.toString, (currTimeInMillis - appInfoStartTime)).stripMargin + submitDate.toString, currTimeInMillis - appInfoStartTime).stripMargin val workerInfoJsonStr = """ From 176e68e6c2e3dc528f51f35719fb6c3160579f2e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Mar 2014 16:04:50 -0800 Subject: [PATCH 51/68] Fix deprecated message for JavaSparkContext (minor) --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index f2e5a854833d9..2a4f471e08c80 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -415,7 +415,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ - @deprecated("added jars are now temporary files and need not be deleted manually", "1.0.0") + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { sc.clearJars() } @@ -424,7 +424,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ - @deprecated("added files are now temporary files and need not be deleted manually", "1.0.0") + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { sc.clearFiles() } From ca258a44af514ac6096ce63bcb28922f8aa4d884 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Mar 2014 17:03:12 -0800 Subject: [PATCH 52/68] Master UI - add support for reading compressed event logs In addition to passing an event log dir to ApplicationDescription, we also pass the compression codec we're using, if we decide to compress logged events. --- .../spark/deploy/ApplicationDescription.scala | 4 ++- .../apache/spark/deploy/master/Master.scala | 25 +++++++++++++------ .../apache/spark/io/CompressionCodec.scala | 5 ++-- .../scheduler/EventLoggingListener.scala | 14 ++++++++++- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 6 ++--- 6 files changed, 41 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 86305d2ea8a09..15fa8a7679874 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy +import org.apache.spark.scheduler.EventLoggingInfo + private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -24,7 +26,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogDir: Option[String] = None) + val eventLogInfo: Option[EventLoggingInfo] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7cc68925a5d2f..864f3d9a908b9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -619,11 +619,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act waitingApps -= app // If application events are logged, use them to rebuild the UI - val rebuildAppUI = app.desc.eventLogDir.isDefined + val rebuildAppUI = app.desc.eventLogInfo.isDefined if (rebuildAppUI) { - val appName = app.desc.name - val eventLogDir = app.desc.eventLogDir.get - val ui = startPersistedSparkUI(appName, eventLogDir) + val ui = startPersistedSparkUI(app) app.desc.appUiUrl = ui.appUIAddress appIdToUI(app.id) = ui } else { @@ -655,9 +653,22 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } } - /** Start a new SparkUI rendered from persisted storage */ - def startPersistedSparkUI(appName: String, eventLogDir: String): SparkUI = { - val ui = new SparkUI(conf, nextPersistedUIPort) + /** + * Start a new SparkUI rendered from persisted storage. Assumes event logging information + * is available for given application. + */ + def startPersistedSparkUI(app: ApplicationInfo): SparkUI = { + val appName = app.desc.name + val eventLogInfo = app.desc.eventLogInfo.get + val eventLogDir = eventLogInfo.logDir + val eventCompressionCodec = eventLogInfo.compressionCodec + val appConf = new SparkConf + eventCompressionCodec.foreach { codec => + appConf.set("spark.eventLog.compress", "true") + appConf.set("spark.io.compression.codec", codec) + } + + val ui = new SparkUI(appConf, nextPersistedUIPort) ui.setAppName(appName) ui.bind() ui.start() diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 848b5c439bb5b..059e58824c39b 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -38,8 +38,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get( - "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) + createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { @@ -47,6 +46,8 @@ private[spark] object CompressionCodec { .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } + + val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName } 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 3a92c1a2c6d2b..859ae74c23835 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,8 +19,9 @@ package org.apache.spark.scheduler import org.json4s.jackson.JsonMethods._ -import org.apache.spark.util.{JsonProtocol, FileLogger} import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, FileLogger} /** * A SparkListener that logs events to persistent storage. @@ -45,6 +46,14 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val logger = new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + // Information needed to replay the events logged by this listener later + val info = { + val compressionCodec = if (shouldCompress) { + Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) + } else None + EventLoggingInfo(logDir, compressionCodec) + } + logInfo("Logging events to %s".format(logDir)) /** Log the event as JSON */ @@ -77,3 +86,6 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) def stop() = logger.stop() } + +// If compression is not enabled, compressionCodec is None +private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index f490d35ac6b78..395a160cc3f37 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.ui.eventLogDir) + sparkHome, sc.ui.appUIAddress, sc.ui.eventLogInfo) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() 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 b8a5d2dfc3332..656e678cd0f7f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.{SparkConf, Logging, SparkContext, SparkEnv} -import org.apache.spark.scheduler.{SparkReplayerBus, EventLoggingListener} +import org.apache.spark.scheduler.{SparkReplayerBus, EventLoggingListener, EventLoggingInfo} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI @@ -77,8 +77,8 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf, port: Int) e def setAppName(name: String) = appName = name - // Path to directory in which events are logged, if any - def eventLogDir: Option[String] = eventLogger.map { l => Some(l.logDir) }.getOrElse(None) + // Information needed to replay the events logged by this UI, if any + def eventLogInfo: Option[EventLoggingInfo] = eventLogger.map { l => Some(l.info) }.getOrElse(None) /** Bind the HTTP server which backs this web interface */ def bind() { From d59da5f87bbe860cc7273115997cf528e0be2db0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 7 Mar 2014 11:00:29 -0800 Subject: [PATCH 53/68] Avoid logging all the blocks on each executor SparkListenerExecutorsStateChange is refactored into two events: SparkListenerBlockManagerGained and SparkListenerBlockManagerLost. Both of these convey the minimum amount of information needed to reconstruct the storage status (i.e. the BlockManagerId, and in the registration case, the maximum memory associated with the block manager). Further, each executor state change no longer involves logging storage statuses for ALL executors, when only one has been updated. --- .../scala/org/apache/spark/SparkContext.scala | 14 +++++-- .../scala/org/apache/spark/SparkEnv.scala | 14 ++++--- .../apache/spark/scheduler/DAGScheduler.scala | 12 +----- .../org/apache/spark/scheduler/EventBus.scala | 36 +++++++--------- .../scheduler/EventLoggingListener.scala | 4 +- .../spark/scheduler/SparkListener.scala | 15 +++++-- .../spark/storage/BlockManagerMaster.scala | 14 ++++--- .../storage/BlockManagerMasterActor.scala | 12 ++++-- ...scala => BlockManagerStatusListener.scala} | 41 ++++++++++++------ .../apache/spark/storage/StorageUtils.scala | 2 +- .../org/apache/spark/ui/UISparkListener.scala | 19 +++++++-- .../apache/spark/ui/jobs/ExecutorTable.scala | 4 +- .../spark/ui/jobs/JobProgressListener.scala | 39 ++++++++++------- .../apache/spark/ui/jobs/JobProgressUI.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 42 ++++++++++++------- 15 files changed, 167 insertions(+), 105 deletions(-) rename core/src/main/scala/org/apache/spark/storage/{BlockManagerRegistrationListener.scala => BlockManagerStatusListener.scala} (53%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 73488e7758a77..ddcb2d29dc8b0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -208,7 +208,8 @@ class SparkContext( @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() - postEnvironmentUpdateEvent() + postEnvironmentUpdate() + listenForBlockManagerUpdates() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -642,7 +643,7 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) - postEnvironmentUpdateEvent() + postEnvironmentUpdate() } def addSparkListener(listener: SparkListener) { @@ -791,7 +792,7 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } - postEnvironmentUpdateEvent() + postEnvironmentUpdate() } /** @@ -1039,7 +1040,7 @@ class SparkContext( private[spark] def newRddId(): Int = nextRddId.getAndIncrement() /** Post the environment update event if the listener bus is ready */ - private def postEnvironmentUpdateEvent() { + private def postEnvironmentUpdate() { Option(listenerBus).foreach { bus => val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq @@ -1051,6 +1052,11 @@ class SparkContext( } } + /** Start listening for block manager status update events */ + private def listenForBlockManagerUpdates() { + env.blockManager.master.listener.map(_.setListenerBus(listenerBus)) + } + /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index da684d81848bd..d4229a22a7b03 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -167,17 +167,19 @@ object SparkEnv extends Logging { } } - // Listen for block manager registration - val blockManagerListener = new BlockManagerRegistrationListener + val blockManagerStatusListener = new BlockManagerStatusListener + + // Lazy because an akka actor cannot be instantiated outside of Props lazy val blockManagerMasterActor = { val actor = new BlockManagerMasterActor(isLocal, conf) - actor.registerListener(blockManagerListener) + actor.registerListener(blockManagerStatusListener) actor } - val blockManagerMaster = - new BlockManagerMaster(registerOrLookup("BlockManagerMaster", blockManagerMasterActor), conf) - blockManagerMaster.registrationListener = Some(blockManagerListener) + val blockManagerMaster = new BlockManagerMaster( + registerOrLookup("BlockManagerMaster", blockManagerMasterActor), + conf, + blockManagerStatusListener) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) 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 59770f3566c93..9b3350e66d8d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -150,9 +150,6 @@ class DAGScheduler( } } })) - - // Start listening for block manager registration - blockManagerMaster.registrationListener.foreach(_.setListenerBus(listenerBus)) } // Called by TaskScheduler to report task's starting. @@ -194,7 +191,7 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { if (!cacheLocs.contains(rdd.id)) { - val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId] + val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) cacheLocs(rdd.id) = blockIds.map { id => locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) @@ -973,11 +970,6 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } - // Block manager master actor should not be null except during tests - if (blockManagerMaster.driverActor != null) { - val storageStatusList = blockManagerMaster.getStorageStatus - listenerBus.post(SparkListenerExecutorsStateChange(storageStatusList)) - } } private def handleExecutorGained(execId: String, host: String) { @@ -986,8 +978,6 @@ class DAGScheduler( logInfo("Host gained which was in lost list earlier: " + host) failedEpoch -= execId } - // Do not trigger SparkListenerExecutorsStateChange, because it is already triggered in - // blockManagerMaster.registrationListener when a new BlockManager registers with the master } private def handleJobCancellation(jobId: Int) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala index f2c1863a6e9e4..a98ec5f05710c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala @@ -36,38 +36,30 @@ private[spark] trait EventBus { /** * Post an event to all attached listeners. Return true if the shutdown event is posted. */ - protected def postToAll(event: SparkListenerEvent): Boolean = { - postToListeners(event, sparkListeners) - } - - /** - * Post an event to a given list of listeners. Return true if the shutdown event is posted. - */ - protected def postToListeners( - event: SparkListenerEvent, - listeners: Seq[SparkListener]): Boolean = { - + def postToAll(event: SparkListenerEvent): Boolean = { event match { case stageSubmitted: SparkListenerStageSubmitted => - listeners.foreach(_.onStageSubmitted(stageSubmitted)) + sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) case stageCompleted: SparkListenerStageCompleted => - listeners.foreach(_.onStageCompleted(stageCompleted)) + sparkListeners.foreach(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => - listeners.foreach(_.onJobStart(jobStart)) + sparkListeners.foreach(_.onJobStart(jobStart)) case jobEnd: SparkListenerJobEnd => - listeners.foreach(_.onJobEnd(jobEnd)) + sparkListeners.foreach(_.onJobEnd(jobEnd)) case taskStart: SparkListenerTaskStart => - listeners.foreach(_.onTaskStart(taskStart)) + sparkListeners.foreach(_.onTaskStart(taskStart)) case taskGettingResult: SparkListenerTaskGettingResult => - listeners.foreach(_.onTaskGettingResult(taskGettingResult)) + sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => - listeners.foreach(_.onTaskEnd(taskEnd)) + sparkListeners.foreach(_.onTaskEnd(taskEnd)) case environmentUpdate: SparkListenerEnvironmentUpdate => - listeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) - case executorsStateChange: SparkListenerExecutorsStateChange => - listeners.foreach(_.onExecutorsStateChange(executorsStateChange)) + sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case blockManagerGained: SparkListenerBlockManagerGained => + sparkListeners.foreach(_.onBlockManagerGained(blockManagerGained)) + case blockManagerLost: SparkListenerBlockManagerLost => + sparkListeners.foreach(_.onBlockManagerLost(blockManagerLost)) case unpersistRDD: SparkListenerUnpersistRDD => - listeners.foreach(_.onUnpersistRDD(unpersistRDD)) + sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) case SparkListenerShutdown => return true case _ => 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 859ae74c23835..f562aede34640 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -79,7 +79,9 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onJobEnd(event: SparkListenerJobEnd) = logEvent(event, flushLogger = true) - override def onExecutorsStateChange(event: SparkListenerExecutorsStateChange) = + override def onBlockManagerGained(event: SparkListenerBlockManagerGained) = + logEvent(event, flushLogger = true) + override def onBlockManagerLost(event: SparkListenerBlockManagerLost) = logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) 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 8674af928e370..36873bfa90ba8 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.Map import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.storage.StorageStatus +import org.apache.spark.storage.{BlockManagerId, StorageStatus} sealed trait SparkListenerEvent @@ -48,9 +48,11 @@ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkLi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -case class SparkListenerExecutorsStateChange(storageStatusList: Seq[StorageStatus]) +case class SparkListenerBlockManagerGained(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +case class SparkListenerBlockManagerLost(blockManagerId: BlockManagerId) extends SparkListenerEvent + case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -103,9 +105,14 @@ trait SparkListener { def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } /** - * Called when a new executor has joined, or an existing executor is lost + * Called when a new block manager has joined + */ + def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { } + + /** + * Called when an existing block manager has been lost */ - def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { } + def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) { } /** * Called when an RDD is manually unpersisted by the application diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index f33a0f8e511de..95e1ecb797440 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,7 +28,14 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { +class BlockManagerMaster( + var driverActor: ActorRef, + conf: SparkConf, + val listener: Option[BlockManagerStatusListener] = None) + extends Logging { + + def this(driverActor: ActorRef, conf: SparkConf, listener: BlockManagerStatusListener) = + this(driverActor, conf, Some(listener)) val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) @@ -37,8 +44,6 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log val timeout = AkkaUtils.askTimeout(conf) - var registrationListener: Option[BlockManagerRegistrationListener] = None - /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { tell(RemoveExecutor(execId)) @@ -55,8 +60,7 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log } /** Register the BlockManager's id with the driver. */ - def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) logInfo("Registered BlockManager") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index f854674a18aa9..f35b002de61cf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -29,6 +29,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.scheduler.{SparkListenerBlockManagerGained, SparkListenerBlockManagerLost} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -50,7 +51,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) - private val listeners = new ArrayBuffer[BlockManagerRegistrationListener] + private val listeners = new ArrayBuffer[BlockManagerStatusListener] val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong @@ -69,7 +70,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act super.preStart() } - def registerListener(listener: BlockManagerRegistrationListener) = listeners += listener + def registerListener(listener: BlockManagerStatusListener) { + listeners += listener + } def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => @@ -164,6 +167,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockLocations.remove(locations) } } + val blockManagerLost = SparkListenerBlockManagerLost(blockManagerId) + listeners.foreach(_.onBlockManagerLost(blockManagerLost)) } private def expireDeadHosts() { @@ -240,7 +245,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } - listeners.foreach(_.onBlockManagerRegister(storageStatus)) + val blockManagerGained = SparkListenerBlockManagerGained(id, maxMemSize) + listeners.foreach(_.onBlockManagerGained(blockManagerGained)) } private def updateBlockInfo( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala similarity index 53% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala index 4643ae29baa17..0a88ea79b2774 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerRegistrationListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala @@ -17,12 +17,24 @@ package org.apache.spark.storage -import org.apache.spark.scheduler._ -import scala.collection.mutable.ArrayBuffer import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -/** A listener for block manager registration */ -private[spark] class BlockManagerRegistrationListener { +import org.apache.spark.scheduler._ + +/** + * A listener for BlockManager status updates. + * + * This listener provides a way to post executor storage status information as soon as it + * is available (i.e. immediately after the associated BlockManager has registered with the + * driver). This is necessary because the SparkContext is only notified when an executor is + * launched, but by then the storage information is not ready yet. + * + * Further, it is possible for a BlockManager be registered before the listener bus on the + * driver is initialized (e.g. the driver's own BlockManager), in which case the corresponding + * event should be buffered. + */ +private[spark] class BlockManagerStatusListener extends SparkListener { private var _listenerBus: Option[SparkListenerBus] = None @@ -31,21 +43,24 @@ private[spark] class BlockManagerRegistrationListener { with mutable.SynchronizedBuffer[SparkListenerEvent] /** - * Set the listener bus. If there are buffered events, post them all to the listener bus at once. + * Set the listener bus. If there are buffered events, post them all to the listener bus. */ def setListenerBus(listenerBus: SparkListenerBus) = { _listenerBus = Some(listenerBus) - bufferedEvents.map(listenerBus.post) + bufferedEvents.map(listenerBus.postToAll) } /** - * Called when a new BlockManager is registered with the master. If the listener bus is ready, - * post the event; otherwise, buffer it. + * Post the event if the listener bus is ready; otherwise, buffer it. */ - def onBlockManagerRegister(storageStatus: Array[StorageStatus]) { - val executorsStateChange = SparkListenerExecutorsStateChange(storageStatus) - _listenerBus.map(_.post(executorsStateChange)).getOrElse { - bufferedEvents += executorsStateChange - } + private def postOrBuffer(event: SparkListenerEvent) { + _listenerBus.map(_.post(event)).getOrElse { bufferedEvents += event } } + + override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) = + postOrBuffer(blockManagerGained) + + override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) = + postOrBuffer(blockManagerLost) + } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 94ac8caeacb24..07e0ec1ffbced 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -28,7 +28,7 @@ private[spark] class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, - val blocks: mutable.Map[BlockId, BlockStatus]) { + val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) diff --git a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala index 61ecf0e5e882d..405144bbbf9f0 100644 --- a/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/UISparkListener.scala @@ -28,7 +28,9 @@ private[ui] trait UISparkListener extends SparkListener * A SparkListener that maintains executor storage status */ private[ui] class StorageStatusSparkListener extends UISparkListener { - var storageStatusList = Seq[StorageStatus]() + val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + + def storageStatusList = executorIdToStorageStatus.values.toSeq /** Update storage status list to reflect updated block statuses */ def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { @@ -68,8 +70,17 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { updateStorageStatus(unpersistRDD.rddId) } - override def onExecutorsStateChange(executorsStateChange: SparkListenerExecutorsStateChange) { - storageStatusList = executorsStateChange.storageStatusList + override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { + val blockManagerId = blockManagerGained.blockManagerId + val executorId = blockManagerId.executorId + val maxMem = blockManagerGained.maxMem + val storageStatus = new StorageStatus(blockManagerId, maxMem) + executorIdToStorageStatus(executorId) = storageStatus + } + + override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) { + val executorId = blockManagerLost.blockManagerId.executorId + executorIdToStorageStatus.remove(executorId) } /** @@ -81,6 +92,7 @@ private[ui] class StorageStatusSparkListener extends UISparkListener { protected def formatExecutorId(execId: String): String = { if (execId == "localhost") "" else execId } + } /** @@ -120,4 +132,5 @@ private[ui] class RDDInfoSparkListener extends StorageStatusSparkListener { super.onUnpersistRDD(unpersistRDD) updateRDDInfo() } + } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 49581bc6beb1e..c186c9fda9d21 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -56,9 +56,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { private def createExecutorTable() : Seq[Node] = { // Make an executor-id -> address map val executorIdToAddress = mutable.HashMap[String, String]() - val storageStatusList = listener.storageStatusList - for (statusId <- 0 until storageStatusList.size) { - val blockManagerId = storageStatusList(statusId).blockManagerId + listener.blockManagerIds.foreach { blockManagerId => val address = blockManagerId.hostPort val executorId = blockManagerId.executorId executorIdToAddress.put(executorId, address) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index e191883e433b7..f80576e0bcf48 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -19,11 +19,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{ListBuffer, HashMap} -import org.apache.spark.{ExceptionFailure, SparkContext, Success} +import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler._ -import org.apache.spark.ui.StorageStatusSparkListener +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.ui.UISparkListener /** * Tracks task-level information to be displayed in the UI. @@ -32,21 +33,12 @@ import org.apache.spark.ui.StorageStatusSparkListener * class, since the UI thread and the DAGScheduler event loop may otherwise * be reading/updating the internal data structures concurrently. */ -private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) - extends StorageStatusSparkListener { +private[ui] class JobProgressListener(conf: SparkConf) extends UISparkListener { import JobProgressListener._ // How many stages to remember - val retainedStages = if (live) { - sc.conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) - } else { - DEFAULT_RETAINED_STAGES - } - - val stageIdToPool = new HashMap[Int, String]() - val stageIdToDescription = new HashMap[Int, String]() - val poolToActiveStages = new HashMap[String, HashMap[Int, StageInfo]]() + val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) val activeStages = HashMap[Int, StageInfo]() val completedStages = ListBuffer[StageInfo]() @@ -67,9 +59,16 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) val stageIdToTasksFailed = HashMap[Int, Int]() val stageIdToTaskInfos = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() + val stageIdToPool = HashMap[Int, String]() + val stageIdToDescription = HashMap[Int, String]() + val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() + + val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() var schedulingMode: Option[SchedulingMode] = None + def blockManagerIds = executorIdToBlockManagerId.values.toSeq + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo val stageId = stage.stageId @@ -232,7 +231,7 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) } } - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) = { + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { val schedulingModeName = environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") schedulingMode = schedulingModeName match { @@ -240,6 +239,18 @@ private[ui] class JobProgressListener(sc: SparkContext, live: Boolean) case None => None } } + + override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { + val blockManagerId = blockManagerGained.blockManagerId + val executorId = blockManagerId.executorId + executorIdToBlockManagerId(executorId) = blockManagerId + } + + override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) { + val executorId = blockManagerLost.blockManagerId.executorId + executorIdToBlockManagerId.remove(executorId) + } + } private[ui] case class TaskUIData( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 09b820c765149..8f4172dd8ff09 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -22,6 +22,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.SparkUI @@ -42,7 +43,8 @@ private[ui] class JobProgressUI(parent: SparkUI) { private var _listener: Option[JobProgressListener] = None def start() { - _listener = Some(new JobProgressListener(sc, live)) + val conf = if (live) sc.conf else new SparkConf + _listener = Some(new JobProgressListener(conf)) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) 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 99e7c4b6d97c6..fc9a2b7e4658a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -57,8 +57,10 @@ private[spark] object JsonProtocol { jobEndToJson(jobEnd) case environmentUpdate: SparkListenerEnvironmentUpdate => environmentUpdateToJson(environmentUpdate) - case executorsStateChange: SparkListenerExecutorsStateChange => - executorsStateChangeToJson(executorsStateChange) + case blockManagerGained: SparkListenerBlockManagerGained => + blockManagerGainedToJson(blockManagerGained) + case blockManagerLost: SparkListenerBlockManagerLost => + blockManagerLostToJson(blockManagerLost) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) case SparkListenerShutdown => @@ -137,12 +139,17 @@ private[spark] object JsonProtocol { ("Classpath Entries" -> classpathEntries) } - def executorsStateChangeToJson(executorsStateChange: SparkListenerExecutorsStateChange) - : JValue = { - val storageStatusList = - JArray(executorsStateChange.storageStatusList.map(storageStatusToJson).toList) - ("Event" -> Utils.getFormattedClassName(executorsStateChange)) ~ - ("Storage Status List" -> storageStatusList) + def blockManagerGainedToJson(blockManagerGained: SparkListenerBlockManagerGained): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerGained.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerGained)) ~ + ("Block Manager ID" -> blockManagerId) ~ + ("Maximum Memory" -> blockManagerGained.maxMem) + } + + def blockManagerLostToJson(blockManagerLost: SparkListenerBlockManagerLost): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerLost.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerLost)) ~ + ("Block Manager ID" -> blockManagerId) } def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { @@ -382,7 +389,8 @@ private[spark] object JsonProtocol { val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) - val executorsStateChanged = Utils.getFormattedClassName(SparkListenerExecutorsStateChange) + val blockManagerGained = Utils.getFormattedClassName(SparkListenerBlockManagerGained) + val blockManagerLost = Utils.getFormattedClassName(SparkListenerBlockManagerLost) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) @@ -395,7 +403,8 @@ private[spark] object JsonProtocol { case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) case `environmentUpdate` => environmentUpdateFromJson(json) - case `executorsStateChanged` => executorsStateChangeFromJson(json) + case `blockManagerGained` => blockManagerGainedFromJson(json) + case `blockManagerLost` => blockManagerLostFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) case `shutdown` => SparkListenerShutdown } @@ -454,10 +463,15 @@ private[spark] object JsonProtocol { SparkListenerEnvironmentUpdate(environmentDetails) } - def executorsStateChangeFromJson(json: JValue): SparkListenerExecutorsStateChange = { - val storageStatusList = - (json \ "Storage Status List").extract[List[JValue]].map(storageStatusFromJson) - SparkListenerExecutorsStateChange(storageStatusList) + def blockManagerGainedFromJson(json: JValue): SparkListenerBlockManagerGained = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + val maxMem = (json \ "Maximum Memory").extract[Long] + SparkListenerBlockManagerGained(blockManagerId, maxMem) + } + + def blockManagerLostFromJson(json: JValue): SparkListenerBlockManagerLost = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + SparkListenerBlockManagerLost(blockManagerId) } def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { From b6eaea77c52d82012fc32059940fccb845c9f04e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 10 Mar 2014 15:48:57 -0700 Subject: [PATCH 54/68] Treating SparkUI as a handler of MasterUI The main purpose for this is to avoid starting a new Jetty server for each reconstructed SparkUI. This involves refactoring the existing way of organizing handlers. In particular, we currently use an immutable HandlerList to group all handlers belonging to the same UI. However, this commit requires us to attach handlers dynamically to a server after it has already started. A further complication is that the simple HandlerCollection, which can be mutable, does not perform longest prefix matching, such that a new context can be engulfed by an existing one (which is extremely difficult to debug, as the Jetty API is just absolutely superb). With this commit, attached SparkUIs no longer need to start their own servers, but simply reside under the /history prefix of the Master Web UI. --- .../apache/spark/deploy/master/Master.scala | 58 ++++++-------- .../spark/deploy/master/ui/MasterWebUI.scala | 48 ++++++----- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../org/apache/spark/ui/JettyUtils.scala | 80 +++++++++++++------ .../scala/org/apache/spark/ui/SparkUI.scala | 71 +++++++++------- .../org/apache/spark/ui/UIReloader.scala | 5 +- .../scala/org/apache/spark/ui/UIUtils.scala | 39 +++++---- .../apache/spark/ui/env/EnvironmentUI.scala | 9 ++- .../apache/spark/ui/exec/ExecutorsUI.scala | 10 ++- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 9 ++- .../apache/spark/ui/jobs/JobProgressUI.scala | 8 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 8 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 8 +- .../org/apache/spark/ui/jobs/StagePage.scala | 10 ++- .../org/apache/spark/ui/jobs/StageTable.scala | 12 ++- .../spark/ui/storage/BlockManagerUI.scala | 6 +- .../apache/spark/ui/storage/IndexPage.scala | 7 +- .../org/apache/spark/ui/storage/RDDPage.scala | 6 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- docs/configuration.md | 7 -- 21 files changed, 236 insertions(+), 171 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 864f3d9a908b9..0e3bd94a6f58e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -51,9 +51,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") - val MAX_NUM_PERSISTED_UI = conf.getInt("spark.persisted.ui.maxConcurrent", 25) - val PERSISTED_SPARK_UI_PORT = - conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] @@ -68,7 +65,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] - var nextPersistedUIPort = PERSISTED_SPARK_UI_PORT val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -112,7 +108,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() + webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -616,28 +612,18 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act }) completedApps.trimStart(toRemove) } + completedApps += app // Remember it in our history waitingApps -= app // If application events are logged, use them to rebuild the UI - val rebuildAppUI = app.desc.eventLogInfo.isDefined - if (rebuildAppUI) { - val ui = startPersistedSparkUI(app) - app.desc.appUiUrl = ui.appUIAddress + startPersistedSparkUI(app).map { ui => + app.desc.appUiUrl = ui.basePath + webUi.attachUI(ui) appIdToUI(app.id) = ui - } else { - // Avoid broken links + }.getOrElse { + // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } - completedApps += app - - // Cap the number of UIs concurrently running - if (appIdToUI.size > MAX_NUM_PERSISTED_UI) { - val oldCompletedApp = completedApps.find { oldApp => appIdToUI.contains(oldApp.id) } - oldCompletedApp.foreach { oldApp => - completedApps -= oldApp - appIdToUI.remove(oldApp.id).foreach(_.stop()) - } - } for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) @@ -654,12 +640,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } /** - * Start a new SparkUI rendered from persisted storage. Assumes event logging information - * is available for given application. + * Start a new SparkUI rendered from persisted storage. If unsuccessful for any reason, + * return None. Otherwise return the reconstructed UI. */ - def startPersistedSparkUI(app: ApplicationInfo): SparkUI = { + def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.get + val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } val eventLogDir = eventLogInfo.logDir val eventCompressionCodec = eventLogInfo.compressionCodec val appConf = new SparkConf @@ -667,14 +653,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act appConf.set("spark.eventLog.compress", "true") appConf.set("spark.io.compression.codec", codec) } - - val ui = new SparkUI(appConf, nextPersistedUIPort) - ui.setAppName(appName) - ui.bind() + val ui = new SparkUI(appConf, appName, "/history/%s".format(app.id)) + // Do not call ui.bind() to avoid creating a new server for each application ui.start() - ui.renderFromPersistedStorage(eventLogDir) - nextPersistedUIPort += 1 - ui + val success = ui.renderFromPersistedStorage(eventLogDir) + if (!success) { + ui.stop() + None + } else Some(ui) } /** Generate a new app ID given a app's submission date */ @@ -760,9 +746,11 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) - : (ActorSystem, Int, Int) = - { + def startSystemAndActor( + host: String, + port: Int, + webUiPort: Int, + conf: SparkConf): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) val timeout = AkkaUtils.askTimeout(conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 5ab13e7aa6b1f..5d72f369919ad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -23,7 +23,7 @@ import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.{JettyUtils, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -32,21 +32,34 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout(master.conf) - val host = Utils.localHostName() - val port = requestedPort + private val host = Utils.localHostName() + private val port = requestedPort val masterActorRef = master.self - + val timeout = AkkaUtils.askTimeout(master.conf) var server: Option[Server] = None var boundPort: Option[Int] = None - val applicationPage = new ApplicationPage(this) - val indexPage = new IndexPage(this) + private val applicationPage = new ApplicationPage(this) + private val indexPage = new IndexPage(this) + + private val handlers: Seq[(String, Handler)] = { + master.masterMetricsSystem.getServletHandlers ++ + master.applicationMetricsSystem.getServletHandlers ++ + Seq[(String, Handler)]( + ("/static", JettyUtils.createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), + ("/app", (request: HttpServletRequest) => applicationPage.render(request)), + ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), + ("/", (request: HttpServletRequest) => indexPage.render(request)) + ) + } - def start() { + private val rootHandler = JettyUtils.createContextHandlerCollection(handlers) + + def bind() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, rootHandler) server = Some(srv) boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) @@ -57,16 +70,13 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } - val metricsHandlers = master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers - - val handlers = metricsHandlers ++ Array[(String, Handler)]( - ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), - ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), - ("/app", (request: HttpServletRequest) => applicationPage.render(request)), - ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), - ("*", (request: HttpServletRequest) => indexPage.render(request)) - ) + def attachUI(ui: SparkUI) { + val childHandler = ui.rootHandler + rootHandler.addHandler(childHandler) + if (!childHandler.isStarted) { + childHandler.start() + } + } def stop() { server.foreach(_.stop()) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index bdf126f93abc8..ecece6fa89894 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -51,7 +51,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I ("/log", (request: HttpServletRequest) => log(request)), ("/logPage", (request: HttpServletRequest) => logPage(request)), ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), - ("*", (request: HttpServletRequest) => indexPage.render(request)) + ("/", (request: HttpServletRequest) => indexPage.render(request)) ) def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 8f990b55c585e..96a55cc500eff 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -27,7 +27,7 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.eclipse.jetty.server.{Handler, Request, Server} -import org.eclipse.jetty.server.handler.{AbstractHandler, ContextHandler, HandlerList, ResourceHandler} +import org.eclipse.jetty.server.handler._ import org.eclipse.jetty.util.thread.QueuedThreadPool import org.apache.spark.Logging @@ -55,29 +55,31 @@ private[spark] object JettyUtils extends Logging { extractFn: T => String = (in: Any) => in.toString): Handler = { new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, - response: HttpServletResponse) { + def handle( + target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { response.setContentType("%s;charset=utf-8".format(contentType)) response.setStatus(HttpServletResponse.SC_OK) baseRequest.setHandled(true) val result = responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.getWriter().println(extractFn(result)) + response.getWriter.println(extractFn(result)) } } } /** Creates a handler that always redirects the user to a given path */ - def createRedirectHandler(newPath: String): Handler = { + def createRedirectHandler(newPath: String, basePath: String = ""): Handler = { new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, - response: HttpServletResponse) { + def handle( + target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { response.setStatus(302) - response.setHeader("Location", baseRequest.getRootURL + newPath) + response.setHeader("Location", baseRequest.getRootURL + basePath + newPath) baseRequest.setHandled(true) } } @@ -95,9 +97,38 @@ private[spark] object JettyUtils extends Logging { staticHandler } + /** Creates a context handler from the given path */ + def createContextHandler(path: String, handler: Handler): ContextHandler = { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler + } + + /** + * Creates a ContextHandlerCollection from the given context handler representations. + * + * This is a mutable collection of context handlers that performs longest prefix matching + * to decide which handler to direct the request to. This allows us to add a new context + * handler whose path is within the prefix of an existing handler. To add a handler: + * + * contextHandlerCollection.addHandler(contextHandler) + * contextHandler.start() + * + * The second line is necessary only if the context handler is not attached to any server + * that has already started. + */ + def createContextHandlerCollection(handlers: Seq[(String, Handler)]): ContextHandlerCollection = { + val contextHandlers = handlers.map { case (path, handler) => + createContextHandler(path, handler) + } + val contextHandlerCollection = new ContextHandlerCollection + contextHandlerCollection.setHandlers(contextHandlers.toArray) + contextHandlerCollection + } + /** - * Attempts to start a Jetty server at the supplied hostName:port which uses the supplied - * handlers. + * Attempts to start a Jetty server bound to the supplied hostName:port using the given + * context handlers. * * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. @@ -106,23 +137,24 @@ private[spark] object JettyUtils extends Logging { hostName: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + val contextHandlerCollection = createContextHandlerCollection(handlers) + startJettyServer(hostName, port, contextHandlerCollection) + } - val handlersToRegister = handlers.map { case(path, handler) => - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister.toArray) - + /** + * Attempts to start a Jetty server bound to the supplied hostName:port using the given handler. + * + * If the desired port number is contented, continues incrementing ports until a free port is + * found. Returns the chosen port and the jetty Server object. + */ + def startJettyServer(hostName: String, port: Int, handler: Handler): (Server, Int) = { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) - server.setHandler(handlerList) + server.setHandler(handler) Try { server.start() } match { case s: Success[_] => 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 656e678cd0f7f..1b519465ca3b4 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -18,9 +18,10 @@ package org.apache.spark.ui import org.eclipse.jetty.server.{Handler, Server} +import org.eclipse.jetty.server.handler.ContextHandlerCollection -import org.apache.spark.{SparkConf, Logging, SparkContext, SparkEnv} -import org.apache.spark.scheduler.{SparkReplayerBus, EventLoggingListener, EventLoggingInfo} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.scheduler.{EventLoggingListener, EventLoggingInfo, SparkReplayerBus} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI @@ -29,22 +30,23 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark. */ -private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf, port: Int) extends Logging { - - def this(sc: SparkContext) = - this(sc, sc.conf, sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt) - - // Persisted UI constructors - def this(conf: SparkConf, port: Int) = this(null, conf, port) - def this(conf: SparkConf) = - this(conf, conf.get("spark.persisted.ui.port", SparkUI.DEFAULT_PERSISTED_PORT).toInt) - def this() = this(new SparkConf()) +private[spark] class SparkUI( + val sc: SparkContext, + conf: SparkConf, + val appName: String, + val basePath: String = "") + extends Logging { + + def this(sc: SparkContext) = this(sc, sc.conf, sc.appName) + def this(conf: SparkConf, appName: String) = this(null, conf, appName) + def this(conf: SparkConf, appName: String, basePath: String) = + this(null, conf, appName, basePath) // If SparkContext is not provided, assume this UI is rendered from persisted storage val live = sc != null - var appName = if (live) sc.appName else "" private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt private var boundPort: Option[Int] = None private var server: Option[Server] = None private var started = false @@ -54,20 +56,32 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf, port: Int) e private val env = new EnvironmentUI(this) private val exec = new ExecutorsUI(this) - private val handlers = Seq[(String, Handler)]( - ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("/", createRedirectHandler("/stages")) - ) - - // Add MetricsServlet handlers by default - private val metricsServletHandlers = if (live) { - SparkEnv.get.metricsSystem.getServletHandlers - } else { - Array[(String, Handler)]() + private val handlers: Seq[(String, Handler)] = { + val metricsServletHandlers = if (live) { + SparkEnv.get.metricsSystem.getServletHandlers + } else { + Array[(String, Handler)]() + } + storage.getHandlers ++ + jobs.getHandlers ++ + env.getHandlers ++ + exec.getHandlers ++ + metricsServletHandlers ++ + Seq[(String, Handler)]( + ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), + ("/", createRedirectHandler("/stages", basePath)) + ) } - private val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ - exec.getHandlers ++ metricsServletHandlers ++ handlers + // The root handler that encapsulates all children handlers of this UI + val rootHandler: ContextHandlerCollection = { + val prefixedHandlers = handlers.map { case (relativePath, handler) => + JettyUtils.createContextHandler(basePath + relativePath.stripSuffix("/"), handler) + } + val collection = new ContextHandlerCollection + collection.setHandlers(prefixedHandlers.toArray) + collection + } // Only log events if this SparkUI is live private var eventLogger: Option[EventLoggingListener] = None @@ -75,15 +89,14 @@ private[spark] class SparkUI(val sc: SparkContext, conf: SparkConf, port: Int) e // Only replay events if this SparkUI is not live private var replayerBus: Option[SparkReplayerBus] = None - def setAppName(name: String) = appName = name - // Information needed to replay the events logged by this UI, if any - def eventLogInfo: Option[EventLoggingInfo] = eventLogger.map { l => Some(l.info) }.getOrElse(None) + def eventLogInfo: Option[EventLoggingInfo] = + eventLogger.map { l => Some(l.info) }.getOrElse(None) /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer(host, port, rootHandler) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) server = Some(srv) boundPort = Some(usedPort) diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala index c77fcff561ad9..d2863a4033e1c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala @@ -29,10 +29,9 @@ object UIReloader { System.exit(1) } - val port = if (args.length == 2) args(1).toInt else 14040 val conf = new SparkConf() - val ui = new SparkUI(conf, port) - ui.setAppName("Reloaded Application") + conf.set("spark.ui.port", "14040") + val ui = new SparkUI(conf, "Reloaded Application") ui.bind() ui.start() val success = ui.renderFromPersistedStorage(args(0)) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index c1f31cedd8bf0..aba2088d5c6b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -27,44 +27,51 @@ private[spark] object UIUtils { private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). getOrElse("") - def prependBaseUri(resource: String = "") = uiRoot + resource + def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], appName: String, title: String, page: Page.Value) - : Seq[Node] = { + def headerSparkPage( + content: => Seq[Node], + basePath: String, + appName: String, + title: String, + page: Page.Value) : Seq[Node] = { val jobs = page match { - case Stages =>
  • Stages
  • - case _ =>
  • Stages
  • + case Stages => +
  • Stages
  • + case _ =>
  • Stages
  • } val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • + case Storage => +
  • Storage
  • + case _ =>
  • Storage
  • } val environment = page match { case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • +
  • Environment
  • + case _ =>
  • Environment
  • } val executors = page match { case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • +
  • Executors
  • + case _ =>
  • Executors
  • } - - - + + {appName} - {title} ; - UIUtils.headerSparkPage(content, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage( + content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) } /** Header fields for the executors table */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index c186c9fda9d21..0ae32c9b56283 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils /** Page showing executor summary */ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private def listener = parent.listener + private lazy val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 129a5f5bd3112..5846090d4ee6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -26,11 +26,12 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ private[ui] class IndexPage(parent: JobProgressUI) { - private lazy val appName = parent.appName - private lazy val isFairScheduler = parent.isFairScheduler - private lazy val listener = parent.listener + private val appName = parent.appName + private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -90,7 +91,7 @@ private[ui] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 8f4172dd8ff09..a283294558f61 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -30,13 +30,15 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressUI(parent: SparkUI) { - lazy val appName = parent.appName - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - lazy val listener = _listener.get + val appName = parent.appName + val basePath = parent.basePath val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val live = parent.live val sc = parent.sc + lazy val listener = _listener.get + lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 9ea72d6b473f4..eddb024d07b7b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,10 +26,11 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressUI) { - private lazy val appName = parent.appName - private lazy val listener = parent.listener + private val appName = parent.appName + private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -48,7 +49,8 @@ private[ui] class PoolPage(parent: JobProgressUI) { val content =

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 45e647da9968a..6dbd8a54615a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -25,8 +25,9 @@ import org.apache.spark.ui.UIUtils /** Table showing list of pools */ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { + private val basePath = parent.basePath private val poolToActiveStages = listener.poolToActiveStages - private def listener = parent.listener + private lazy val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -61,7 +62,10 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { }
    - {p.name} + + {p.name} + {p.minShare} {p.weight}{s.stageId} - + {poolName.get}
    - + {rdd.name}
    spark.persisted.ui.port14040 - Port on which a Spark UI rebuilt from persisted storage is hosted. -
    spark.ui.retainedStages 1000
    {k}{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}{parent.formatDuration(v.taskTime)}{v.failedTasks + v.succeededTasks}{v.failedTasks}{v.succeededTasks}{Utils.bytesToString(v.shuffleRead)}{Utils.bytesToString(v.shuffleWrite)}{Utils.bytesToString(v.memoryBytesSpilled)}{Utils.bytesToString(v.diskBytesSpilled)}
    {k}{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}{parent.formatDuration(v.taskTime)}{v.failedTasks + v.succeededTasks}{v.failedTasks}{v.succeededTasks}{Utils.bytesToString(v.shuffleRead)}{Utils.bytesToString(v.shuffleWrite)}{Utils.bytesToString(v.memoryBytesSpilled)}{Utils.bytesToString(v.diskBytesSpilled)}
    @@ -62,8 +61,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index ae5d38f5086fc..b2732de51058a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -33,7 +33,6 @@ private[ui] class IndexPage(parent: BlockManagerUI) { private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - // Calculate macro-level statistics val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 1eaf5e6ecec00..0d76fb57c7603 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -18,14 +18,14 @@ package org.apache.spark.util import java.io._ -import java.text.SimpleDateFormat import java.net.URI +import java.text.SimpleDateFormat import java.util.Date import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import org.apache.hadoop.fs.{FSDataOutputStream, Path} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec /** @@ -38,7 +38,7 @@ import org.apache.spark.io.CompressionCodec */ class FileLogger( logDir: String, - conf: SparkConf = new SparkConf(), + conf: SparkConf = new SparkConf, outputBufferSize: Int = 8 * 1024, compress: Boolean = false, overwrite: Boolean = true) 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 fc9a2b7e4658a..80dcd7a6de3ae 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -17,17 +17,17 @@ package org.apache.spark.util -import java.util.{UUID, Properties} +import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable +import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ -import org.json4s.DefaultFormats -import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ @@ -36,7 +36,7 @@ private[spark] object JsonProtocol { private implicit val format = DefaultFormats /** - * JSON serialization methods for SparkListenerEvent's + * JSON serialization methods for SparkListenerEvents */ def sparkEventToJson(event: SparkListenerEvent): JValue = { @@ -162,7 +162,7 @@ private[spark] object JsonProtocol { } /** - * JSON serialization methods for classes SparkListenerEvent's depend on + * JSON serialization methods for classes SparkListenerEvents depend on */ def stageInfoToJson(stageInfo: StageInfo): JValue = { @@ -377,7 +377,7 @@ private[spark] object JsonProtocol { } /** - * JSON deserialization methods for SparkListenerEvent's + * JSON deserialization methods for SparkListenerEvents */ def sparkEventFromJson(json: JValue): SparkListenerEvent = { @@ -479,7 +479,7 @@ private[spark] object JsonProtocol { } /** - * JSON deserialization methods for classes SparkListenerEvent's depend on + * JSON deserialization methods for classes SparkListenerEvents depend on */ def stageInfoFromJson(json: JValue): StageInfo = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d61b2b3e4ff55..75f1565d8e221 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.io._ -import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL} +import java.net.{Inet4Address, InetAddress, NetworkInterface, URI, URL} import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} @@ -31,14 +31,13 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.json4s._ import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.json4s._ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - /** * Various utility methods used by Spark. */ diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 77ecea4f709f5..7f6ebc4255c75 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -23,8 +23,8 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { @@ -55,7 +55,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar expecting { blockManager.get(RDDBlockId(0, 0)).andReturn(None) blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andStubReturn(Seq()) + true).andStubReturn(Seq[(BlockId, BlockStatus)]()) } whenExecuting(blockManager) { diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 713ef69f35edb..7a39d1af9e2d5 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ -import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 0e44e41813d77..4e5bd035041a8 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskScheduler} +import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 0c23495819111..9f2924c23b73c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -20,12 +20,9 @@ package org.apache.spark.deploy import java.io.File import java.util.Date +import com.fasterxml.jackson.core.JsonParseException import org.json4s._ - -import org.json4s.JValue import org.json4s.jackson.JsonMethods -import com.fasterxml.jackson.core.JsonParseException - import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} 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 e5dfa6acfa40f..c97543f57d8f3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -94,8 +94,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) - scheduler = new DAGScheduler(taskScheduler, sc.listenerBus, mapOutputTracker, - blockManagerMaster, sc.env) { + scheduler = new DAGScheduler( + taskScheduler, + sc.listenerBus, + mapOutputTracker, + blockManagerMaster, + sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing runLocallyWithinThread(job) 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 bade1cbb70b77..d8a3e859f85cd 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 @@ -19,9 +19,9 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite -import org.apache.spark.scheduler._ import org.apache.spark.{LocalSparkContext, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.scheduler._ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { From ac69ec897ba47acc1a8ffe7a94c03d1ab185e313 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 11 Mar 2014 20:10:48 -0700 Subject: [PATCH 57/68] Fix test fail --- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 96a6edfa0e0d1..dbdabfc5283db 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -47,7 +47,8 @@ class UISuite extends FunSuite { } test("jetty binds to port 0 correctly") { - val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq(), new SparkConf) + val (jettyServer, boundPort, _) = JettyUtils.startJettyServer( + "0.0.0.0", 0, Seq[ServletContextHandler](), new SparkConf) assert(jettyServer.getState === "STARTED") assert(boundPort != 0) Try {new ServerSocket(boundPort)} match { From bf80e3db07d700cdec1a6e6f00ae3a7b96c074ff Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 11 Mar 2014 20:39:16 -0700 Subject: [PATCH 58/68] Imports, comments, and code formatting, once again (minor) --- core/src/main/scala/org/apache/spark/SecurityManager.scala | 1 + .../scala/org/apache/spark/scheduler/SparkListener.scala | 2 +- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 2 +- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../main/scala/org/apache/spark/ui/jobs/PoolTable.scala | 7 ++++--- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 2 +- 6 files changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index da8ff43996fda..2237ee3bb7aad 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -22,6 +22,7 @@ import java.net.{Authenticator, PasswordAuthentication} import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.io.Text + import org.apache.spark.deploy.SparkHadoopUtil /** 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 02a68146a8b0c..8ab2b8d3b275c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -21,10 +21,10 @@ import java.util.Properties import scala.collection.Map -import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.{Distribution, Utils} sealed trait SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 394d9da0ac559..f525a934cac6a 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -180,7 +180,7 @@ private[spark] object JettyUtils extends Logging { * context handlers. * * If the desired port number is contended, continues incrementing ports until a free port is - * found. Returns the jetty Server object, the chosen port, and a mutable collection of handlers. + * found. Return the jetty Server object, the chosen port, and a mutable collection of handlers. */ def startJettyServer( hostName: String, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 7cd4183d51889..fb42258ba7501 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -143,7 +143,7 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { // create executor summary map if necessary val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, op = new HashMap[String, ExecutorSummary]()) - executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary()) + executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary) val executorSummary = executorSummaryMap.get(info.executorId) executorSummary match { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index f565fa96d273b..c5c8d8668740b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -37,7 +37,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { private def poolTable( makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], - rows: Seq[Schedulable]) : Seq[Node] = { + rows: Seq[Schedulable]): Seq[Node] = {
    Pool Name
    - + {p.name}
    @@ -53,8 +53,9 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) {
    Pool Name
    } - private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]) - : Seq[Node] = { + private def poolRow( + p: Schedulable, + poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]): Seq[Node] = { val activeStages = poolToActiveStages.get(p.name) match { case Some(stages) => stages.size case None => 0 diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index dbdabfc5283db..7f88a9bfb5161 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -22,10 +22,10 @@ import java.net.ServerSocket import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite import org.apache.spark.SparkConf -import org.eclipse.jetty.servlet.ServletContextHandler class UISuite extends FunSuite { test("jetty port increases under contention") { From 3456090b6fa959825421bbb40ad5f5b0f2e0df0a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 13 Mar 2014 14:32:10 -0700 Subject: [PATCH 59/68] Address Patrick's comments --- .../scheduler/EventLoggingListener.scala | 4 ++-- .../spark/scheduler/SparkListenerBus.scala | 12 +++++++---- .../spark/scheduler/SparkReplayerBus.scala | 5 +++-- .../org/apache/spark/util/FileLogger.scala | 9 ++++---- .../scala/org/apache/spark/util/Utils.scala | 21 ++----------------- 5 files changed, 20 insertions(+), 31 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 f562aede34640..6ad5b46ba1d3f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -37,10 +37,10 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) - private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", true) + private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") - private val name = appName.replaceAll("[ /]", "-").toLowerCase + "-" + System.currentTimeMillis() + private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis() val logDir = logBaseDir + "/" + name private val logger = diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index f6cb3702f8bef..a49b8daa5b489 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -24,8 +24,10 @@ import org.apache.spark.Logging /** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ private[spark] class SparkListenerBus extends EventBus with Logging { - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + /** + * Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. + */ private val EVENT_QUEUE_CAPACITY = 10000 private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false @@ -67,8 +69,10 @@ private[spark] class SparkListenerBus extends EventBus with Logging { if (System.currentTimeMillis > finishTime) { return false } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ + /** + * Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. + */ Thread.sleep(10) } true diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala index 63db674cb0599..b1973f72dcdee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.InputStream +import java.net.URI import scala.io.Source @@ -59,7 +60,7 @@ private[spark] class SparkReplayerBus(conf: SparkConf) extends EventBus with Log * Replay each event in the order maintained in the given logs. */ def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(logDir) + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) val logPaths = getLogFilePaths(logDir, fileSystem) if (logPaths.length == 0) { return false @@ -84,7 +85,7 @@ private[spark] class SparkReplayerBus(conf: SparkConf) extends EventBus with Log } } catch { case e: Exception => - logWarning("Exception in parsing UI logs for %s".format(path)) + logWarning("Exception in parsing Spark event log %s".format(path)) logWarning(currentLine + "\n") logDebug(e.getMessage + e.getStackTraceString) } finally { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0d76fb57c7603..43421b77a7525 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -45,7 +45,7 @@ class FileLogger( extends Logging { private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val fileSystem = Utils.getHadoopFileSystem(logDir) + private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) private var fileIndex = 0 // Only used if compression is enabled @@ -63,15 +63,16 @@ class FileLogger( private def createLogDir() { val path = new Path(logDir) if (fileSystem.exists(path)) { - logWarning("Log directory already exists.") if (overwrite) { + logWarning("Log directory %s already exists. Overwriting...".format(logDir)) // Second parameter is whether to delete recursively fileSystem.delete(path, true) + } else { + throw new IOException("Log directory %s already exists!".format(logDir)) } } if (!fileSystem.mkdirs(path)) { - // Logger should throw a exception rather than continue to construct this object - throw new IOException("Error in creating log directory:" + logDir) + throw new IOException("Error in creating log directory: %s".format(logDir)) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 91f82a9cc0d16..26be4bfdae1c2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -282,7 +282,7 @@ private[spark] object Utils extends Logging { uc.setConnectTimeout(timeout) uc.setReadTimeout(timeout) uc.connect() - val in = uc.getInputStream(); + val in = uc.getInputStream() val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { @@ -911,25 +911,8 @@ private[spark] object Utils extends Logging { /** * Return a Hadoop FileSystem with the scheme encoded in the given path. - * File systems currently supported include HDFS, S3, and the local file system. */ def getHadoopFileSystem(path: URI): FileSystem = { - path.getScheme match { - case "file" | "hdfs" | "s3" | null => - val conf = SparkHadoopUtil.get.newConfiguration() - FileSystem.get(path, conf) - case unsupportedScheme => - throw new UnsupportedOperationException("File system scheme %s is not supported!" - .format(unsupportedScheme)) - } - } - - /** - * Return a Hadoop FileSystem with the scheme encoded in the given path. - * File systems currently supported include HDFS, S3, and the local file system. - */ - def getHadoopFileSystem(path: String): FileSystem = { - val uri = new URI(path) - getHadoopFileSystem(uri) + FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } } From c5c2c8f04eda980feeacdcd345c71a145c06e8af Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 13 Mar 2014 16:48:10 -0700 Subject: [PATCH 60/68] Remove list of (TaskInfo, TaskMetrics) from StageInfo From an experiment, I discovered that up to 38% of the logged bytes is made up of StageInfo (compared to 45% for TaskEnd, which is unavoidable). This is because the StageInfo in the StageCompleted events currently store lists of (TaskInfo, TaskMetrics) objects, which are duplicated in TaskEnd events anyway. This commit gets rid of this list, which significantly cuts down on log size and thus log time. --- .../apache/spark/scheduler/DAGScheduler.scala | 1 - .../scheduler/EventLoggingListener.scala | 2 +- .../spark/scheduler/SparkListener.scala | 76 +++++++++------ .../apache/spark/scheduler/StageInfo.scala | 14 +-- .../org/apache/spark/util/JsonProtocol.scala | 12 +-- .../spark/scheduler/SparkListenerSuite.scala | 92 +++++++++++-------- 6 files changed, 105 insertions(+), 92 deletions(-) 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 3e5ecc0f9968d..ed5b8a31a2a67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -818,7 +818,6 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { 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 6ad5b46ba1d3f..82a9407bc93ce 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -40,7 +40,7 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") - private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis() + private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis val logDir = logBaseDir + "/" + name private val logger = 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 8ab2b8d3b275c..e434e4dce4fa8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import scala.collection.Map +import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics @@ -129,25 +130,36 @@ trait SparkListener { * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { + private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) - showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) + showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) // Shuffle write showBytesDistribution("shuffle bytes written:", - (_,metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten)) + (_, metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten), taskInfoMetrics) // Fetch & I/O showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime)) + (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead)) - showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) + (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) + showBytesDistribution("task result size:", + (_, metric) => Some(metric.resultSize), taskInfoMetrics) // Runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfos.map { case (info, metrics) => + val runtimePcts = taskInfoMetrics.map { case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", @@ -155,6 +167,7 @@ class StatsReportListener extends SparkListener with Logging { showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") + taskInfoMetrics.clear() } } @@ -166,18 +179,19 @@ private[spark] object StatsReportListener extends Logging { val probabilities = percentiles.map(_ / 100.0) val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - def extractDoubleDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo, TaskMetrics) => Option[Double]) - : Option[Distribution] = { - Distribution(stage.stageInfo.taskInfos.flatMap { - case ((info,metric)) => getMetric(info, metric)}) + def extractDoubleDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) } // Is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - : Option[Distribution] = { - extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map(_.toDouble)) + def extractLongDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution( + taskInfoMetrics, + (info, metric) => { getMetric(info, metric).map(_.toDouble) }) } def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { @@ -188,31 +202,35 @@ private[spark] object StatsReportListener extends Logging { logInfo("\t" + quantiles.mkString("\t")) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) - { + def showDistribution( + heading: String, + dOpt: Option[Distribution], + formatNumber: Double => String) { dOpt.foreach { d => showDistribution(heading, d, formatNumber)} } def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { - def f(d:Double) = format.format(d) + def f(d: Double) = format.format(d) showDistribution(heading, dOpt, f _) } def showDistribution( heading: String, format: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Double]) - (implicit stage: SparkListenerStageCompleted) { - showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) + getMetric: (TaskInfo, TaskMetrics) => Option[Double], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) } - def showBytesDistribution(heading:String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) + def showBytesDistribution( + heading:String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - dOpt.foreach{dist => showBytesDistribution(heading, dist)} + dOpt.foreach { dist => showBytesDistribution(heading, dist) } } def showBytesDistribution(heading: String, dist: Distribution) { @@ -224,9 +242,11 @@ private[spark] object StatsReportListener extends Logging { (d => StatsReportListener.millisToString(d.toLong)): Double => String) } - def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) + def showMillisDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } val seconds = 1000L diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 3dfc1af2892ae..8115a7ed7896d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,23 +17,13 @@ package org.apache.spark.scheduler -import scala.collection.mutable - -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.RDDInfo /** - * Stores information about a stage to pass from the scheduler to SparkListeners. Also - * stores the metrics for all tasks that have completed, including redundant, speculated tasks. + * Stores information about a stage to pass from the scheduler to SparkListeners. */ private[spark] -class StageInfo( - val stageId: Int, - val name: String, - val numTasks: Int, - val rddInfo: RDDInfo, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer.empty) { - +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None 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 80dcd7a6de3ae..f0b9b1bbbc310 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -167,19 +167,12 @@ private[spark] object JsonProtocol { def stageInfoToJson(stageInfo: StageInfo): JValue = { val rddInfo = rddInfoToJson(stageInfo.rddInfo) - val taskInfos = JArray(stageInfo.taskInfos.map { case (info, metrics) => - val metricsJson = if (metrics != null) taskMetricsToJson(metrics) else JNothing - val infoJson = if (info != null) taskInfoToJson(info) else JNothing - ("Task Info" -> infoJson) ~ - ("Task Metrics" -> metricsJson) - }.toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ - ("Task Infos" -> taskInfos) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) @@ -487,14 +480,11 @@ private[spark] object JsonProtocol { val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfo = rddInfoFromJson(json \ "RDD Info") - val taskInfos = (json \ "Task Infos").extract[List[JValue]].map { value => - (taskInfoFromJson(value \ "Task Info"), taskMetricsFromJson(value \ "Task Metrics")) - }.toBuffer val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo, taskInfos) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index d80ac007b1612..cbbd987d2da3f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Buffer, HashSet} +import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { @@ -39,42 +40,42 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } test("basic creation of StageInfo") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") rdd2.count assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.rddInfo.name should be {"Target RDD"} - first.numTasks should be {4} - first.rddInfo.numPartitions should be {4} - first.submissionTime should be ('defined) - first.completionTime should be ('defined) - first.taskInfos.length should be {4} + val (stageInfo, taskInfoMetrics) = listener.stageInfos.head + stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.numTasks should be {4} + stageInfo.rddInfo.numPartitions should be {4} + stageInfo.submissionTime should be ('defined) + stageInfo.completionTime should be ('defined) + taskInfoMetrics.length should be {4} } test("StageInfo with fewer tasks than partitions") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1), true) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.numTasks should be {2} - first.rddInfo.numPartitions should be {4} + val (stageInfo, _) = listener.stageInfos.head + stageInfo.numTasks should be {2} + stageInfo.rddInfo.numPartitions should be {4} } test("local metrics") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) //just to make sure some of the tasks take a noticeable amount of time @@ -84,39 +85,39 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc i } - val d = sc.parallelize(0 to 1e4.toInt, 64).map{i => w(i)} + val d = sc.parallelize(0 to 1e4.toInt, 64).map(w) d.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (1) - val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1") - - val d3 = d.map{i => w(i) -> (0 to (i % 5))}.setName("shuffle input 2") - - val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)} + val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") + val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2") + val d4 = d2.cogroup(d3, 64).map { case (k, (v1, v2)) => + w(k) -> (v1.size, v2.size) + } d4.setName("A Cogroup") - d4.collectAsMap() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (4) - listener.stageInfos.foreach { stageInfo => - /* small test, so some tasks might take less than 1 millisecond, but average should be greater - * than 0 ms. */ - checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") + listener.stageInfos.foreach { case (stageInfo, taskInfoMetrics) => + /** + * Small test, so some tasks might take less than 1 millisecond, but average should be greater + * than 0 ms. + */ checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorRunTime}, + taskInfoMetrics.map(_._2.executorRunTime), stageInfo + " executorRunTime") checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorDeserializeTime}, + taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") if (stageInfo.rddInfo.name == d4.name) { checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, + taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") } - stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => + taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) @@ -142,7 +143,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x) + val result = sc.parallelize(Seq(1), 1) + .map { x => 1.to(akkaFrameSize).toArray } + .reduce { case (x, y) => x } assert(result === 1.to(akkaFrameSize).toArray) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -157,7 +160,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) // Make a task whose result is larger than the akka frame size - val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) + val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -204,17 +207,28 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc assert(m.sum / m.size.toDouble > 0.0, msg) } - class SaveStageInfo extends SparkListener { - val stageInfos = Buffer[StageInfo]() + class SaveStageAndTaskInfo extends SparkListener { + val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() + var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(task: SparkListenerTaskEnd) { + val info = task.taskInfo + val metrics = task.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stage: SparkListenerStageCompleted) { - stageInfos += stage.stageInfo + stageInfos(stage.stageInfo) = taskInfoMetrics + taskInfoMetrics = mutable.Buffer.empty } } class SaveTaskEvents extends SparkListener { - val startedTasks = new HashSet[Int]() - val startedGettingResultTasks = new HashSet[Int]() - val endedTasks = new HashSet[Int]() + val startedTasks = new mutable.HashSet[Int]() + val startedGettingResultTasks = new mutable.HashSet[Int]() + val endedTasks = new mutable.HashSet[Int]() override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { startedTasks += taskStart.taskInfo.index From 45fd84c838aab5b51ff7e6eee56670c5d1f73bea Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 13 Mar 2014 17:03:09 -0700 Subject: [PATCH 61/68] Remove now deprecated test --- .../spark/scheduler/JobLoggerSuite.scala | 121 ------------------ 1 file changed, 121 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala deleted file mode 100644 index 012f6ef4de3f0..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - -import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { - val WAIT_TIMEOUT_MILLIS = 10000 - - test("inner method") { - sc = new SparkContext("local", "joblogger") - val joblogger = new JobLogger { - def createLogWriterTest(jobId: Int) = createLogWriter(jobId) - def closeLogWriterTest(jobId: Int) = closeLogWriter(jobId) - } - type MyRDD = RDD[(Int, Int)] - def makeRdd(numPartitions: Int, dependencies: List[Dependency[_]]): MyRDD = { - val maxPartition = numPartitions - 1 - new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - } - } - val jobId = 5 - val parentRdd = makeRdd(4, Nil) - val shuffleDep = new ShuffleDependency(parentRdd, null) - val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = - new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobId, None) - val rootStage = - new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobId, None) - val rootStageInfo = StageInfo.fromStage(rootStage) - - joblogger.onJobStart(SparkListenerJobStart(jobId, Seq[Int](0, 1))) - joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo)) - joblogger.createLogWriterTest(jobId) - joblogger.getJobIdToPrintWriter.size should be (1) - joblogger.getJobIdToStageIds.get(jobId).get.size should be (2) - joblogger.getStageIdToJobId.get(0) should be (Some(jobId)) - joblogger.getStageIdToJobId.get(1) should be (Some(jobId)) - joblogger.closeLogWriterTest(jobId) - joblogger.getStageIdToJobId.size should be (0) - joblogger.getJobIdToStageIds.size should be (0) - joblogger.getJobIdToPrintWriter.size should be (0) - } - - test("inner variables") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - override protected def closeLogWriter(jobId: Int) = - getJobIdToPrintWriter.get(jobId).foreach { fileWriter => - fileWriter.close() - } - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) - - joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIdToPrintWriter.size should be (1) - joblogger.getStageIdToJobId.size should be (2) - joblogger.getStageIdToJobId.get(0) should be (Some(0)) - joblogger.getStageIdToJobId.get(1) should be (Some(0)) - joblogger.getJobIdToStageIds.size should be (1) - } - - - test("interface functions") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - var onTaskEndCount = 0 - var onJobEndCount = 0 - var onJobStartCount = 0 - var onStageCompletedCount = 0 - var onStageSubmittedCount = 0 - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 - override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 - override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - joblogger.onJobStartCount should be (1) - joblogger.onJobEndCount should be (1) - joblogger.onTaskEndCount should be (8) - joblogger.onStageSubmittedCount should be (2) - joblogger.onStageCompletedCount should be (2) - } -} From 650eb12c09fdf351bab7dfda91f87c7af99f74c8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Mar 2014 21:06:40 -0700 Subject: [PATCH 62/68] Add unit tests + Fix bugs found through tests This covers all JSON de/serialization logic and block manager reporting blocks with updated storage statuses during put. --- .../apache/spark/storage/BlockManager.scala | 33 +- .../apache/spark/storage/MemoryStore.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 27 +- .../spark/storage/BlockManagerSuite.scala | 65 +- .../apache/spark/util/JsonProtocolSuite.scala | 564 ++++++++++++++++++ 5 files changed, 647 insertions(+), 44 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 806e8524c7561..8d4a122f55051 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -268,7 +268,9 @@ private[spark] class BlockManager( case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) - val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) + val deserialized = if (inMem) level.deserialized else false + val replication = if (inMem || onDisk) level.replication else 1 + val storageLevel = StorageLevel(onDisk, inMem, deserialized, replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L (storageLevel, memSize, diskSize) @@ -498,8 +500,11 @@ private[spark] class BlockManager( * This is currently used for writing shuffle files out. Callers should handle error * cases. */ - def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) - : BlockObjectWriter = { + def getDiskWriter( + blockId: BlockId, + file: File, + serializer: Serializer, + bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) @@ -639,15 +644,17 @@ private[spark] class BlockManager( } } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. - marked = true - putBlockInfo.markReady(size) val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - if (tellMaster) { - reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + if (putBlockStatus.storageLevel != StorageLevel.NONE) { + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + marked = true + putBlockInfo.markReady(size) + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + } + updatedBlocks += ((blockId, putBlockStatus)) } - updatedBlocks += ((blockId, putBlockStatus)) } finally { // If we failed in putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. @@ -731,7 +738,11 @@ private[spark] class BlockManager( /** * Write a block consisting of a single object. */ - def putSingle(blockId: BlockId, value: Any, level: StorageLevel, tellMaster: Boolean = true) { + def putSingle( + blockId: BlockId, + value: Any, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { put(blockId, Iterator(value), level, tellMaster) } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 0ea3f53112060..049f09044c0d8 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -49,7 +49,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() 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 f0b9b1bbbc310..b3957331e980d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -269,18 +269,6 @@ private[spark] object JsonProtocol { ("Result" -> result) ~ json } - def storageStatusToJson(storageStatus: StorageStatus): JValue = { - val blockManagerId = blockManagerIdToJson(storageStatus.blockManagerId) - val blocks = JArray( - storageStatus.blocks.toList.map { case (id, status) => - ("Block ID" -> blockIdToJson(id)) ~ - ("Status" -> blockStatusToJson(status)) - }) - ("Block Manager ID" -> blockManagerId) ~ - ("Maximum Memory" -> storageStatus.maxMem) ~ - ("Blocks" -> blocks) - } - def rddInfoToJson(rddInfo: RDDInfo): JValue = { val storageLevel = storageLevelToJson(rddInfo.storageLevel) ("RDD ID" -> rddInfo.id) ~ @@ -365,7 +353,7 @@ private[spark] object JsonProtocol { } def exceptionToJson(exception: Exception): JValue = { - ("Message" -> exception.toString) ~ + ("Message" -> exception.getMessage) ~ ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) } @@ -516,6 +504,7 @@ private[spark] object JsonProtocol { metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] + metrics.resultSize = (json \ "Result Size").extract[Long] metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] @@ -605,18 +594,6 @@ private[spark] object JsonProtocol { } } - def storageStatusFromJson(json: JValue): StorageStatus = { - val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") - val maxMem = (json \ "Maximum Memory").extract[Long] - val blocks = (json \ "Blocks").extract[List[JValue]].map { block => - val id = blockIdFromJson(block \ "Block ID") - val status = blockStatusFromJson(block \ "Status") - (id, status) - } - val blockMap = mutable.Map[BlockId, BlockStatus](blocks: _*) - new StorageStatus(blockManagerId, maxMem, blockMap) - } - def rddInfoFromJson(json: JValue): RDDInfo = { val rddId = (json \ "RDD ID").extract[Int] val name = (json \ "Name").extract[String] diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1036b9f34e9dd..cb4b0d197fa06 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -492,12 +492,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + assert(store.getSingle("a3").isDefined, "a3 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") @@ -663,6 +660,60 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("updated block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + val list = List.fill(2)(new Array[Byte](200)) + val bigList = List.fill(8)(new Array[Byte](200)) + + // 1 updated block (i.e. list1) + val updatedBlocks1 = + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks1.size === 1) + assert(updatedBlocks1.head._1 === TestBlockId("list1")) + assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 1 updated block (i.e. list2) + val updatedBlocks2 = + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + assert(updatedBlocks2.size === 1) + assert(updatedBlocks2.head._1 === TestBlockId("list2")) + assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 2 updated blocks - list1 is kicked out of memory while list3 is added + val updatedBlocks3 = + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks3.size === 2) + updatedBlocks3.foreach { case (id, status) => + id match { + case TestBlockId("list1") => assert(status.storageLevel === StorageLevel.NONE) + case TestBlockId("list3") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list1 nor list3") + } + } + assert(store.get("list3").isDefined, "list3 was not in store") + + // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added + val updatedBlocks4 = + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks4.size === 2) + updatedBlocks4.foreach { case (id, status) => + id match { + case TestBlockId("list2") => assert(status.storageLevel === StorageLevel.DISK_ONLY) + case TestBlockId("list4") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list2 nor list4") + } + } + assert(store.get("list4").isDefined, "list4 was not in store") + + // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + val updatedBlocks5 = + store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks5.size === 0) + assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.get("list4").isDefined, "list4 was not in store") + assert(!store.get("list5").isDefined, "list5 was in store") + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala new file mode 100644 index 0000000000000..fb6eb5cd4fe37 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -0,0 +1,564 @@ +/* + * 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.util + +import java.util.{Properties, UUID} + +import scala.collection.Map + +import org.json4s.jackson.JsonMethods._ +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.executor._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +class JsonProtocolSuite extends FunSuite { + + test("SparkListenerEvent") { + val stageSubmitted = + SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) + val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) + val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 444L)) + val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 3000L)) + val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + makeTaskInfo(123L, 234, 345L), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) + val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobEnd = SparkListenerJobEnd(20, JobSucceeded) + val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( + "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), + "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), + "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), + "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) + )) + val blockManagerGained = SparkListenerBlockManagerGained( + BlockManagerId("Stars", "In your multitude...", 300, 400), 500) + val blockManagerLost = SparkListenerBlockManagerLost( + BlockManagerId("Scarce", "to be counted...", 100, 200)) + val unpersistRdd = SparkListenerUnpersistRDD(12345) + + testEvent(stageSubmitted, stageSubmittedJsonString) + testEvent(stageCompleted, stageCompletedJsonString) + testEvent(taskStart, taskStartJsonString) + testEvent(taskGettingResult, taskGettingResultJsonString) + testEvent(taskEnd, taskEndJsonString) + testEvent(jobStart, jobStartJsonString) + testEvent(jobEnd, jobEndJsonString) + testEvent(environmentUpdate, environmentUpdateJsonString) + testEvent(blockManagerGained, blockManagerGainedJsonString) + testEvent(blockManagerLost, blockManagerLostJsonString) + testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(SparkListenerShutdown, shutdownJsonString) + } + + test("Dependent Classes") { + testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) + testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) + testTaskInfo(makeTaskInfo(999L, 888, 777L)) + testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8)) + testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000)) + + // StorageLevel + testStorageLevel(StorageLevel.NONE) + testStorageLevel(StorageLevel.DISK_ONLY) + testStorageLevel(StorageLevel.DISK_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY) + testStorageLevel(StorageLevel.MEMORY_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER_2) + + // JobResult + val exception = new Exception("Out of Memory! Please restock film.") + exception.setStackTrace(stackTrace) + val jobFailed = JobFailed(exception, 2) + testJobResult(JobSucceeded) + testJobResult(jobFailed) + + // TaskEndReason + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15, 16), 17, 18, 19) + val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None) + testTaskEndReason(Success) + testTaskEndReason(Resubmitted) + testTaskEndReason(fetchFailed) + testTaskEndReason(exceptionFailure) + testTaskEndReason(TaskResultLost) + testTaskEndReason(TaskKilled) + testTaskEndReason(ExecutorLostFailure) + testTaskEndReason(UnknownReason) + + // BlockId + testBlockId(RDDBlockId(1, 2)) + testBlockId(ShuffleBlockId(1, 2, 3)) + testBlockId(BroadcastBlockId(1L)) + testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(TaskResultBlockId(1L)) + testBlockId(StreamBlockId(1, 2L)) + testBlockId(TempBlockId(UUID.randomUUID())) + } + + + /** -------------------------- * + | Helper test running methods | + * --------------------------- */ + + private def testEvent(event: SparkListenerEvent, jsonString: String) { + val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) + val newEvent = JsonProtocol.sparkEventFromJson(parse(actualJsonString)) + assertJsonStringEquals(jsonString, actualJsonString) + assertEquals(event, newEvent) + } + + private def testRDDInfo(info: RDDInfo) { + val newInfo = JsonProtocol.rddInfoFromJson(JsonProtocol.rddInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStageInfo(info: StageInfo) { + val newInfo = JsonProtocol.stageInfoFromJson(JsonProtocol.stageInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStorageLevel(level: StorageLevel) { + val newLevel = JsonProtocol.storageLevelFromJson(JsonProtocol.storageLevelToJson(level)) + assertEquals(level, newLevel) + } + + private def testTaskMetrics(metrics: TaskMetrics) { + val newMetrics = JsonProtocol.taskMetricsFromJson(JsonProtocol.taskMetricsToJson(metrics)) + assertEquals(metrics, newMetrics) + } + + private def testBlockManagerId(id: BlockManagerId) { + val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) + assertEquals(id, newId) + } + + private def testTaskInfo(info: TaskInfo) { + val newInfo = JsonProtocol.taskInfoFromJson(JsonProtocol.taskInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testJobResult(result: JobResult) { + val newResult = JsonProtocol.jobResultFromJson(JsonProtocol.jobResultToJson(result)) + assertEquals(result, newResult) + } + + private def testTaskEndReason(reason: TaskEndReason) { + val newReason = JsonProtocol.taskEndReasonFromJson(JsonProtocol.taskEndReasonToJson(reason)) + assertEquals(reason, newReason) + } + + private def testBlockId(blockId: BlockId) { + val newBlockId = JsonProtocol.blockIdFromJson(JsonProtocol.blockIdToJson(blockId)) + blockId == newBlockId + } + + + /** -------------------------------- * + | Util methods for comparing events | + * --------------------------------- */ + + private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { + (event1, event2) match { + case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => + assert(e1.properties == e2.properties) + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerStageCompleted, e2: SparkListenerStageCompleted) => + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerTaskStart, e2: SparkListenerTaskStart) => + assert(e1.stageId == e2.stageId) + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskGettingResult, e2: SparkListenerTaskGettingResult) => + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) => + assert(e1.stageId == e2.stageId) + assert(e1.taskType == e2.taskType) + assertEquals(e1.reason, e2.reason) + assertEquals(e1.taskInfo, e2.taskInfo) + assertEquals(e1.taskMetrics, e2.taskMetrics) + case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => + assert(e1.jobId == e2.jobId) + assert(e1.properties == e2.properties) + assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 == i2)) + case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => + assert(e1.jobId == e2.jobId) + assertEquals(e1.jobResult, e2.jobResult) + case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => + assertEquals(e1.environmentDetails, e2.environmentDetails) + case (e1: SparkListenerBlockManagerGained, e2: SparkListenerBlockManagerGained) => + assert(e1.maxMem == e2.maxMem) + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerBlockManagerLost, e2: SparkListenerBlockManagerLost) => + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => + assert(e1.rddId == e2.rddId) + case (SparkListenerShutdown, SparkListenerShutdown) => + case _ => fail("Events don't match in types!") + } + } + + private def assertEquals(info1: StageInfo, info2: StageInfo) { + assert(info1.stageId == info2.stageId) + assert(info1.name == info2.name) + assert(info1.numTasks == info2.numTasks) + assert(info1.submissionTime == info2.submissionTime) + assert(info1.completionTime == info2.completionTime) + assert(info1.emittedTaskSizeWarning == info2.emittedTaskSizeWarning) + assertEquals(info1.rddInfo, info2.rddInfo) + } + + private def assertEquals(info1: RDDInfo, info2: RDDInfo) { + assert(info1.id == info2.id) + assert(info1.name == info2.name) + assert(info1.numPartitions == info2.numPartitions) + assert(info1.numCachedPartitions == info2.numCachedPartitions) + assert(info1.memSize == info2.memSize) + assert(info1.diskSize == info2.diskSize) + assertEquals(info1.storageLevel, info2.storageLevel) + } + + private def assertEquals(level1: StorageLevel, level2: StorageLevel) { + assert(level1.useDisk == level2.useDisk) + assert(level1.useMemory == level2.useMemory) + assert(level1.deserialized == level2.deserialized) + assert(level1.replication == level2.replication) + } + + private def assertEquals(info1: TaskInfo, info2: TaskInfo) { + assert(info1.taskId == info2.taskId) + assert(info1.index == info2.index) + assert(info1.launchTime == info2.launchTime) + assert(info1.executorId == info2.executorId) + assert(info1.host == info2.host) + assert(info1.taskLocality == info2.taskLocality) + assert(info1.gettingResultTime == info2.gettingResultTime) + assert(info1.finishTime == info2.finishTime) + assert(info1.failed == info2.failed) + assert(info1.serializedSize == info2.serializedSize) + } + + private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { + assert(metrics1.hostname == metrics2.hostname) + assert(metrics1.executorDeserializeTime == metrics2.executorDeserializeTime) + assert(metrics1.resultSize == metrics2.resultSize) + assert(metrics1.jvmGCTime == metrics2.jvmGCTime) + assert(metrics1.resultSerializationTime == metrics2.resultSerializationTime) + assert(metrics1.memoryBytesSpilled == metrics2.memoryBytesSpilled) + assert(metrics1.diskBytesSpilled == metrics2.diskBytesSpilled) + assertOptionEquals( + metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals) + assertOptionEquals( + metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) + assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) + } + + private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { + assert(metrics1.shuffleFinishTime == metrics2.shuffleFinishTime) + assert(metrics1.totalBlocksFetched == metrics2.totalBlocksFetched) + assert(metrics1.remoteBlocksFetched == metrics2.remoteBlocksFetched) + assert(metrics1.localBlocksFetched == metrics2.localBlocksFetched) + assert(metrics1.fetchWaitTime == metrics2.fetchWaitTime) + assert(metrics1.remoteBytesRead == metrics2.remoteBytesRead) + } + + private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { + assert(metrics1.shuffleBytesWritten == metrics2.shuffleBytesWritten) + assert(metrics1.shuffleWriteTime == metrics2.shuffleWriteTime) + } + + private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { + assert(bm1.executorId == bm2.executorId) + assert(bm1.host == bm2.host) + assert(bm1.port == bm2.port) + assert(bm1.nettyPort == bm2.nettyPort) + } + + private def assertEquals(result1: JobResult, result2: JobResult) { + (result1, result2) match { + case (JobSucceeded, JobSucceeded) => + case (r1: JobFailed, r2: JobFailed) => + assert(r1.failedStageId == r2.failedStageId) + assertEquals(r1.exception, r2.exception) + case _ => fail("Job results don't match in types!") + } + } + + private def assertEquals(reason1: TaskEndReason, reason2: TaskEndReason) { + (reason1, reason2) match { + case (Success, Success) => + case (Resubmitted, Resubmitted) => + case (r1: FetchFailed, r2: FetchFailed) => + assert(r1.shuffleId == r2.shuffleId) + assert(r1.mapId == r2.mapId) + assert(r1.reduceId == r2.reduceId) + assertEquals(r1.bmAddress, r2.bmAddress) + case (r1: ExceptionFailure, r2: ExceptionFailure) => + assert(r1.className == r2.className) + assert(r1.description == r2.description) + assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) + assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) + case (TaskResultLost, TaskResultLost) => + case (TaskKilled, TaskKilled) => + case (ExecutorLostFailure, ExecutorLostFailure) => + case (UnknownReason, UnknownReason) => + case _ => fail("Task end reasons don't match in types!") + } + } + + private def assertEquals( + details1: Map[String, Seq[(String, String)]], + details2: Map[String, Seq[(String, String)]]) { + details1.zip(details2).foreach { + case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => + assert(key1 == key2) + values1.zip(values2).foreach { case (v1, v2) => assert(v1 == v2) } + } + } + + private def assertEquals(exception1: Exception, exception2: Exception) { + assert(exception1.getMessage == exception2.getMessage) + assertSeqEquals( + exception1.getStackTrace, + exception2.getStackTrace, + assertStackTraceElementEquals) + } + + private def assertJsonStringEquals(json1: String, json2: String) { + val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") + formatJsonString(json1) == formatJsonString(json2) + } + + private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { + assert(seq1.length == seq2.length) + seq1.zip(seq2).foreach { case (t1, t2) => + assertEquals(t1, t2) + } + } + + private def assertOptionEquals[T]( + opt1: Option[T], + opt2: Option[T], + assertEquals: (T, T) => Unit) { + if (opt1.isDefined) { + assert(opt2.isDefined) + assertEquals(opt1.get, opt2.get) + } else { + assert(!opt2.isDefined) + } + } + + /** + * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals + */ + + private def assertShuffleReadEquals(r1: ShuffleReadMetrics, r2: ShuffleReadMetrics) { + assertEquals(r1, r2) + } + + private def assertShuffleWriteEquals(w1: ShuffleWriteMetrics, w2: ShuffleWriteMetrics) { + assertEquals(w1, w2) + } + + private def assertTaskMetricsEquals(t1: TaskMetrics, t2: TaskMetrics) { + assertEquals(t1, t2) + } + + private def assertBlocksEquals( + blocks1: Seq[(BlockId, BlockStatus)], + blocks2: Seq[(BlockId, BlockStatus)]) = { + assertSeqEquals(blocks1, blocks2, assertBlockEquals) + } + + private def assertBlockEquals(b1: (BlockId, BlockStatus), b2: (BlockId, BlockStatus)) { + assert(b1 == b2) + } + + private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) { + assert(ste1 == ste2) + } + + + /** ----------------------------------- * + | Util methods for constructing events | + * ------------------------------------ */ + + private val properties = { + val p = new Properties + p.setProperty("Ukraine", "Kiev") + p.setProperty("Russia", "Moscow") + p.setProperty("France", "Paris") + p.setProperty("Germany", "Berlin") + p + } + + private val stackTrace = { + Array[StackTraceElement]( + new StackTraceElement("Apollo", "Venus", "Mercury", 42), + new StackTraceElement("Afollo", "Vemus", "Mercurry", 420), + new StackTraceElement("Ayollo", "Vesus", "Blackberry", 4200) + ) + } + + private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK) + r.numCachedPartitions = c + r.memSize = d + r.diskSize = e + r + } + + private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + } + + private def makeTaskInfo(a: Long, b: Int, c: Long) = { + new TaskInfo(a, b, c, "executor", "your kind sir", TaskLocality.NODE_LOCAL) + } + + private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, e: Int, f: Int) = { + val t = new TaskMetrics + val sr = new ShuffleReadMetrics + val sw = new ShuffleWriteMetrics + t.hostname = "localhost" + t.executorDeserializeTime = a + t.executorRunTime = b + t.resultSize = c + t.jvmGCTime = d + t.resultSerializationTime = a + b + t.memoryBytesSpilled = a + c + sr.shuffleFinishTime = b + c + sr.totalBlocksFetched = e + f + sr.remoteBytesRead = b + d + sr.localBlocksFetched = e + sr.fetchWaitTime = a + d + sr.remoteBlocksFetched = f + sw.shuffleBytesWritten = a + b + c + sw.shuffleWriteTime = b + c + d + t.shuffleReadMetrics = Some(sr) + t.shuffleWriteMetrics = Some(sw) + // Make at most 6 blocks + t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => + (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i)) + }.toSeq) + t + } + + + /** --------------------------------------- * + | JSON string representation of each event | + * ---------------------------------------- */ + + private val stageSubmittedJsonString = + """ + {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": + "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, + "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", + "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val stageCompletedJsonString = + """ + {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": + "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, + "Disk Size":501},"Emitted Task Size Warning":false}} + """ + + private val taskStartJsonString = + """ + {"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, + "Index":333,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskGettingResultJsonString = + """ + {"Event":"SparkListenerTaskGettingResult","Task Info":{"Task ID":1000,"Index": + 2000,"Launch Time":3000,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskEndJsonString = + """ + {"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", + "Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index": + 234,"Launch Time":345,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed": + false,"Serialized Size":0},"Task Metrics":{"Host Name":"localhost", + "Executor Deserialize Time":300,"Executor Run Time":400,"Result Size":500, + "JVM GC Time":600,"Result Serialization Time":700,"Memory Bytes Spilled": + 800,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Shuffle Finish Time": + 900,"Total Blocks Fetched":1500,"Remote Blocks Fetched":800,"Local Blocks Fetched": + 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": + {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": + [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + """ + + private val jobStartJsonString = + """ + {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": + {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val jobEndJsonString = + """ + {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}} + """ + + private val environmentUpdateJsonString = + """ + {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s", + "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s, + regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"}, + "Classpath Entries":{"Super library":"/tmp/super_library"}} + """ + + private val blockManagerGainedJsonString = + """ + {"Event":"SparkListenerBlockManagerGained","Block Manager ID":{"Executor ID":"Stars", + "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} + """ + + private val blockManagerLostJsonString = + """ + {"Event":"SparkListenerBlockManagerLost","Block Manager ID":{"Executor ID":"Scarce", + "Host":"to be counted...","Port":100,"Netty Port":200}} + """ + + private val unpersistRDDJsonString = + """ + {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} + """ + + private val shutdownJsonString = + """ + {"Event":"SparkListenerShutdown"} + """ + } From 6740e49df5562ea990df03442c6d5fe57e450f03 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Mar 2014 21:22:59 -0700 Subject: [PATCH 63/68] Fix comment nits --- .../scala/org/apache/spark/CacheManager.scala | 12 +++---- .../spark/scheduler/SparkListenerBus.scala | 12 +++---- .../storage/BlockManagerMasterActor.scala | 10 +++--- .../apache/spark/storage/MemoryStore.scala | 10 +++--- .../org/apache/spark/util/FileLogger.scala | 6 ++-- .../org/apache/spark/util/JsonProtocol.scala | 36 +++++++++---------- 6 files changed, 38 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 220d6114d965e..5f50a5dd6e414 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -50,13 +50,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { try {loading.wait()} catch {case _ : Throwable =>} } logInfo("Finished waiting for %s".format(key)) - /** - * See whether someone else has successfully loaded it. The main way this would fail + /* See whether someone else has successfully loaded it. The main way this would fail * is for the RDD-level cache eviction policy if someone else has loaded the same RDD * partition but we didn't want to make space for it. However, that case is unlikely * because it's unlikely that two threads would work on the same RDD partition. One - * downside of the current code is that threads wait serially if this does happen. - */ + * downside of the current code is that threads wait serially if this does happen. */ blockManager.get(key) match { case Some(values) => return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) @@ -80,15 +78,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { var updatedBlocks = Seq[(BlockId, BlockStatus)]() val returnValue: Iterator[T] = { if (storageLevel.useDisk && !storageLevel.useMemory) { - /** - * In the case that this RDD is to be persisted using DISK_ONLY + /* In the case that this RDD is to be persisted using DISK_ONLY * the iterator will be passed directly to the blockManager (rather then * caching it to an ArrayBuffer first), then the resulting block data iterator * will be passed back to the user. If the iterator generates a lot of data, * this means that it doesn't all have to be held in memory at one time. * This could also apply to MEMORY_ONLY_SER storage, but we need to make sure - * blocks aren't dropped by the block store before enabling that. - */ + * blocks aren't dropped by the block store before enabling that. */ updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) blockManager.get(key) match { case Some(values) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index a49b8daa5b489..f6cb3702f8bef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -24,10 +24,8 @@ import org.apache.spark.Logging /** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ private[spark] class SparkListenerBus extends EventBus with Logging { - /** - * Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. - */ + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ private val EVENT_QUEUE_CAPACITY = 10000 private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false @@ -69,10 +67,8 @@ private[spark] class SparkListenerBus extends EventBus with Logging { if (System.currentTimeMillis > finishTime) { return false } - /** - * Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. - */ + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ Thread.sleep(10) } true diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 1eac0b6863d9c..1dfb6e5c2447d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -355,11 +355,11 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - // isValid means it is either stored in-memory or on-disk. - // But the memSize here indicates the data size in or dropped from memory, - // and the diskSize here indicates the data size in or dropped to disk. - // They can be both larger than 0, when a block is dropped from memory to disk. - // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. + /* isValid means it is either stored in-memory or on-disk. + * But the memSize here indicates the data size in or dropped from memory, + * and the diskSize here indicates the data size in or dropped to disk. + * They can be both larger than 0, when a block is dropped from memory to disk. + * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ if (storageLevel.useMemory) { _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) _remainingMem -= memSize diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 049f09044c0d8..7ed7cdd430e7a 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -166,11 +166,11 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) size: Long, deserialized: Boolean): ResultWithDroppedBlocks = { - // TODO: Its possible to optimize the locking by locking entries only when selecting blocks - // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has - // been released, it must be ensured that those to-be-dropped blocks are not double counted - // for freeing up more space for another block that needs to be put. Only then the actually - // dropping of blocks (and writing to disk if necessary) can proceed in parallel. + /* TODO: Its possible to optimize the locking by locking entries only when selecting blocks + * to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has + * been released, it must be ensured that those to-be-dropped blocks are not double counted + * for freeing up more space for another block that needs to be put. Only then the actually + * dropping of blocks (and writing to disk if necessary) can proceed in parallel. */ var putSuccess = false val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 43421b77a7525..93d5d4a24d689 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -84,10 +84,8 @@ class FileLogger( val logPath = logDir + "/" + fileIndex val uri = new URI(logPath) - /** - * The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. - */ + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ val dstream = uri.getScheme match { case "hdfs" | "s3" => val path = new Path(logPath) 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 b3957331e980d..edf3711d37899 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -35,9 +35,9 @@ import org.apache.spark._ private[spark] object JsonProtocol { private implicit val format = DefaultFormats - /** - * JSON serialization methods for SparkListenerEvents - */ + /** ------------------------------------------------- * + * JSON serialization methods for SparkListenerEvents | + * -------------------------------------------------- */ def sparkEventToJson(event: SparkListenerEvent): JValue = { event match { @@ -161,9 +161,9 @@ private[spark] object JsonProtocol { "Event" -> Utils.getFormattedClassName(SparkListenerShutdown) } - /** - * JSON serialization methods for classes SparkListenerEvents depend on - */ + /** ------------------------------------------------------------------- * + * JSON serialization methods for classes SparkListenerEvents depend on | + * -------------------------------------------------------------------- */ def stageInfoToJson(stageInfo: StageInfo): JValue = { val rddInfo = rddInfoToJson(stageInfo.rddInfo) @@ -323,9 +323,9 @@ private[spark] object JsonProtocol { ("Disk Size" -> blockStatus.diskSize) } - /** - * Util JSON serialization methods - */ + /** ------------------------------ * + * Util JSON serialization methods | + * ------------------------------- */ def mapToJson(m: Map[String, String]): JValue = { val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } @@ -357,9 +357,9 @@ private[spark] object JsonProtocol { ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) } - /** - * JSON deserialization methods for SparkListenerEvents - */ + /** --------------------------------------------------- * + * JSON deserialization methods for SparkListenerEvents | + * ---------------------------------------------------- */ def sparkEventFromJson(json: JValue): SparkListenerEvent = { val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) @@ -459,9 +459,9 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } - /** - * JSON deserialization methods for classes SparkListenerEvents depend on - */ + /** --------------------------------------------------------------------- * + * JSON deserialization methods for classes SparkListenerEvents depend on | + * ---------------------------------------------------------------------- */ def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] @@ -669,9 +669,9 @@ private[spark] object JsonProtocol { BlockStatus(storageLevel, memorySize, diskSize) } - /** - * Util JSON deserialization methods - */ + /** -------------------------------- * + * Util JSON deserialization methods | + * --------------------------------- */ def mapFromJson(json: JValue): Map[String, String] = { val jsonFields = json.asInstanceOf[JObject].obj From 9e14f9714b78eb7cfb4f6372d6d2ee76096e621c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 14 Mar 2014 21:54:18 -0700 Subject: [PATCH 64/68] Moved around functionality + renamed classes per Patrick --- .../scala/org/apache/spark/SparkContext.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 11 +- .../apache/spark/scheduler/DAGScheduler.scala | 14 +- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/EventBus.scala | 69 -------- .../scheduler/EventLoggingListener.scala | 2 +- .../apache/spark/scheduler/JobLogger.scala | 4 +- .../spark/scheduler/LiveListenerBus.scala | 78 ++++++++++ ...layerBus.scala => ReplayListenerBus.scala} | 2 +- .../spark/scheduler/SparkListener.scala | 4 +- .../spark/scheduler/SparkListenerBus.scala | 147 ++++++++---------- .../spark/scheduler/TaskSchedulerImpl.scala | 6 +- .../storage/BlockManagerMasterActor.scala | 6 +- .../storage/BlockManagerStatusListener.scala | 8 +- .../spark/storage/StorageStatusListener.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 58 ++----- .../org/apache/spark/ui/UIReloader.scala | 50 ------ .../spark/ui/jobs/JobProgressListener.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 20 +-- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 10 +- docs/configuration.md | 2 +- 24 files changed, 224 insertions(+), 294 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/EventBus.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala rename core/src/main/scala/org/apache/spark/scheduler/{SparkReplayerBus.scala => ReplayListenerBus.scala} (97%) delete mode 100644 core/src/main/scala/org/apache/spark/ui/UIReloader.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7d07984509bd9..c564760b3010b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -151,10 +151,6 @@ class SparkContext( private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - // Initialize the Spark UI - private[spark] val ui = new SparkUI(this) - ui.bind() - val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -200,9 +196,11 @@ class SparkContext( executorEnvs("SPARK_USER") = sparkUser // An asynchronous listener bus for Spark events - private[spark] val listenerBus = new SparkListenerBus + private[spark] val listenerBus = new LiveListenerBus // Start the UI before posting events to listener bus, because the UI listens for Spark events + private[spark] val ui = new SparkUI(this) + ui.bind() ui.start() // Create and start the scheduler diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 580f53ac0d317..7f49ec96d5e66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -37,6 +37,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.scheduler.ReplayListenerBus import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -666,10 +667,16 @@ private[spark] class Master( appConf.set("spark.eventLog.compress", "true") appConf.set("spark.io.compression.codec", codec) } - val ui = new SparkUI(appConf, "%s (finished)".format(appName), "/history/%s".format(app.id)) + val replayerBus = new ReplayListenerBus(appConf) + val ui = new SparkUI( + appConf, + replayerBus, + "%s (finished)".format(appName), + "/history/%s".format(app.id)) + // Do not call ui.bind() to avoid creating a new server for each application ui.start() - val success = ui.renderFromPersistedStorage(eventLogDir) + val success = replayerBus.replay(eventLogDir) if (!success) { ui.stop() None 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 ed5b8a31a2a67..11a8d848c8e70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -55,7 +55,7 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH private[spark] class DAGScheduler( taskScheduler: TaskScheduler, - listenerBus: SparkListenerBus, + listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv) @@ -179,8 +179,8 @@ class DAGScheduler( } // Called by TaskScheduler when a host is added - def executorGained(execId: String, host: String) { - eventProcessActor ! ExecutorGained(execId, host) + def executorAdded(execId: String, host: String) { + eventProcessActor ! ExecutorAdded(execId, host) } // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or @@ -561,8 +561,8 @@ class DAGScheduler( activeJobs.clear() // These should already be empty by this point, stageIdToActiveJob.clear() // but just in case we lost track of some jobs... - case ExecutorGained(execId, host) => - handleExecutorGained(execId, host) + case ExecutorAdded(execId, host) => + handleExecutorAdded(execId, host) case ExecutorLost(execId) => handleExecutorLost(execId) @@ -971,10 +971,10 @@ class DAGScheduler( } } - private def handleExecutorGained(execId: String, host: String) { + private def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { - logInfo("Host gained which was in lost list earlier: " + host) + logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 39cd98e2d74e4..04c53d468465a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,7 +65,7 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent +private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala b/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala deleted file mode 100644 index 2176fbc11fc55..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/EventBus.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -/** - * A SparkListenerEvent bus that relays events to its listeners - */ -private[spark] trait EventBus { - - // SparkListeners attached to this event bus - protected val sparkListeners = new ArrayBuffer[SparkListener] - with mutable.SynchronizedBuffer[SparkListener] - - def addListener(listener: SparkListener) { - sparkListeners += listener - } - - /** - * Post an event to all attached listeners. Return true if the shutdown event is posted. - */ - def postToAll(event: SparkListenerEvent): Boolean = { - event match { - case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) - case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) - case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) - case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) - case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) - case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) - case environmentUpdate: SparkListenerEnvironmentUpdate => - sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) - case blockManagerGained: SparkListenerBlockManagerGained => - sparkListeners.foreach(_.onBlockManagerGained(blockManagerGained)) - case blockManagerLost: SparkListenerBlockManagerLost => - sparkListeners.foreach(_.onBlockManagerLost(blockManagerLost)) - case unpersistRDD: SparkListenerUnpersistRDD => - sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) - case SparkListenerShutdown => - return true - case _ => - } - false - } -} 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 82a9407bc93ce..4985f8c43e70d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -79,7 +79,7 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onJobEnd(event: SparkListenerJobEnd) = logEvent(event, flushLogger = true) - override def onBlockManagerGained(event: SparkListenerBlockManagerGained) = + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) = logEvent(event, flushLogger = true) override def onBlockManagerLost(event: SparkListenerBlockManagerLost) = logEvent(event, flushLogger = true) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 0778e4820e9b3..b3a67d7e17976 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -35,8 +35,8 @@ import org.apache.spark.executor.TaskMetrics * * NOTE: The functionality of this class is heavily stripped down to accommodate for a general * refactor of the SparkListener interface. In its place, the EventLoggingListener is introduced - * to log application information as SparkListenerEvents through the SparkUI. To enable this - * functionality, set spark.eventLog.enabled to true. + * to log application information as SparkListenerEvents. To enable this functionality, set + * spark.eventLog.enabled to true. */ @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala new file mode 100644 index 0000000000000..71cc47eda33eb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + + // Create a new daemon thread to listen for events. This thread is stopped when it receives + // a SparkListenerShutdown event, using the stop method. + new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + val shutdown = postToAll(event) + if (shutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + } + } + }.start() + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop(): Unit = post(SparkListenerShutdown) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala rename to core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index b1973f72dcdee..e2963b3287b28 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkReplayerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} /** * An EventBus that replays logged events from persisted storage */ -private[spark] class SparkReplayerBus(conf: SparkConf) extends EventBus with Logging { +private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { private val compressed = conf.getBoolean("spark.eventLog.compress", false) // Only used if compression is enabled 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 e434e4dce4fa8..e4391bc56664b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -54,7 +54,7 @@ case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkLi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent -case class SparkListenerBlockManagerGained(blockManagerId: BlockManagerId, maxMem: Long) +case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent case class SparkListenerBlockManagerLost(blockManagerId: BlockManagerId) extends SparkListenerEvent @@ -113,7 +113,7 @@ trait SparkListener { /** * Called when a new block manager has joined */ - def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { } + def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { } /** * Called when an existing block manager has been lost diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index f6cb3702f8bef..67d39c40291f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -1,78 +1,69 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import org.apache.spark.Logging - -/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ -private[spark] class SparkListenerBus extends EventBus with Logging { - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - - // Create a new daemon thread to listen for events. This thread is stopped when it receives - // a SparkListenerShutdown event, using the stop method. - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - val shutdown = postToAll(event) - if (shutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - } - } - }.start() - - def post(event: SparkListenerEvent) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop(): Unit = post(SparkListenerShutdown) -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +/** + * A SparkListenerEvent bus that relays events to its listeners + */ +private[spark] trait SparkListenerBus { + + // SparkListeners attached to this event bus + protected val sparkListeners = new ArrayBuffer[SparkListener] + with mutable.SynchronizedBuffer[SparkListener] + + def addListener(listener: SparkListener) { + sparkListeners += listener + } + + /** + * Post an event to all attached listeners. Return true if the shutdown event is posted. + */ + def postToAll(event: SparkListenerEvent): Boolean = { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + case stageCompleted: SparkListenerStageCompleted => + sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + case jobStart: SparkListenerJobStart => + sparkListeners.foreach(_.onJobStart(jobStart)) + case jobEnd: SparkListenerJobEnd => + sparkListeners.foreach(_.onJobEnd(jobEnd)) + case taskStart: SparkListenerTaskStart => + sparkListeners.foreach(_.onTaskStart(taskStart)) + case taskGettingResult: SparkListenerTaskGettingResult => + sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + case taskEnd: SparkListenerTaskEnd => + sparkListeners.foreach(_.onTaskEnd(taskEnd)) + case environmentUpdate: SparkListenerEnvironmentUpdate => + sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case blockManagerAdded: SparkListenerBlockManagerAdded => + sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + case blockManagerLost: SparkListenerBlockManagerLost => + sparkListeners.foreach(_.onBlockManagerLost(blockManagerLost)) + case unpersistRDD: SparkListenerUnpersistRDD => + sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case SparkListenerShutdown => + return true + case _ => + } + false + } +} 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 23b06612fd7ab..1c5e528800fb4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -204,7 +204,7 @@ private[spark] class TaskSchedulerImpl( executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() - executorGained(o.executorId, o.host) + executorAdded(o.executorId, o.host) } } @@ -399,8 +399,8 @@ private[spark] class TaskSchedulerImpl( rootPool.executorLost(executorId, host) } - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) + def executorAdded(execId: String, host: String) { + dagScheduler.executorAdded(execId, host) } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 1dfb6e5c2447d..1e8edc31ba083 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,7 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} -import org.apache.spark.scheduler.{SparkListenerBlockManagerGained, SparkListenerBlockManagerLost} +import org.apache.spark.scheduler.{SparkListenerBlockManagerAdded, SparkListenerBlockManagerLost} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -245,8 +245,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } - val blockManagerGained = SparkListenerBlockManagerGained(id, maxMemSize) - statusListener.foreach(_.onBlockManagerGained(blockManagerGained)) + val blockManagerAdded = SparkListenerBlockManagerAdded(id, maxMemSize) + statusListener.foreach(_.onBlockManagerAdded(blockManagerAdded)) } private def updateBlockInfo( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala index 9af73d5d88779..be3755d189ccb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala @@ -34,7 +34,7 @@ import org.apache.spark.scheduler._ * event should be buffered. */ private[spark] class BlockManagerStatusListener extends SparkListener { - private var _listenerBus: Option[SparkListenerBus] = None + private var _listenerBus: Option[LiveListenerBus] = None // Buffer any events received before the listener bus is ready private val bufferedEvents = new ArrayBuffer[SparkListenerEvent] @@ -42,7 +42,7 @@ private[spark] class BlockManagerStatusListener extends SparkListener { /** * Set the listener bus. If there are buffered events, post them all to the listener bus. */ - def setListenerBus(listenerBus: SparkListenerBus) = { + def setListenerBus(listenerBus: LiveListenerBus) = { _listenerBus = Some(listenerBus) bufferedEvents.map(listenerBus.postToAll) } @@ -54,8 +54,8 @@ private[spark] class BlockManagerStatusListener extends SparkListener { _listenerBus.map(_.post(event)).getOrElse { bufferedEvents += event } } - override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) = - postOrBuffer(blockManagerGained) + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) = + postOrBuffer(blockManagerAdded) override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) = postOrBuffer(blockManagerLost) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 2aaf02869b18a..fd6b82f4a64d5 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -67,11 +67,11 @@ private[spark] class StorageStatusListener extends SparkListener { updateStorageStatus(unpersistRDD.rddId) } - override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { synchronized { - val blockManagerId = blockManagerGained.blockManagerId + val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId - val maxMem = blockManagerGained.maxMem + val maxMem = blockManagerAdded.maxMem val storageStatus = new StorageStatus(blockManagerId, maxMem) executorIdToStorageStatus(executorId) = storageStatus } 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 53f2eb59c0a7a..bc138c5398eb7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,7 +21,7 @@ import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} -import org.apache.spark.scheduler.{EventLoggingInfo, EventLoggingListener, SparkReplayerBus} +import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI @@ -34,14 +34,14 @@ import org.apache.spark.util.Utils private[spark] class SparkUI( val sc: SparkContext, conf: SparkConf, + val listenerBus: SparkListenerBus, val appName: String, val basePath: String = "") extends Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.appName) - def this(conf: SparkConf, appName: String) = this(null, conf, appName) - def this(conf: SparkConf, appName: String, basePath: String) = - this(null, conf, appName, basePath) + def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null @@ -52,7 +52,6 @@ private[spark] class SparkUI( private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt private var boundPort: Option[Int] = None private var server: Option[Server] = None - private var started = false private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -82,9 +81,6 @@ private[spark] class SparkUI( // Only log events if this SparkUI is live private var eventLogger: Option[EventLoggingListener] = None - // Only replay events if this SparkUI is not live - private var replayerBus: Option[SparkReplayerBus] = None - // Information needed to replay the events logged by this UI, if any lazy val eventLogInfo: Option[EventLoggingInfo] = eventLogger.map { l => Some(l.info) }.getOrElse(None) @@ -110,39 +106,19 @@ private[spark] class SparkUI( env.start() exec.start() - // Listen for events from the SparkContext if it exists, otherwise from persisted storage - val eventBus = if (live) { - val loggingEnabled = conf.getBoolean("spark.eventLog.enabled", false) - if (loggingEnabled) { - val logger = new EventLoggingListener(appName, conf) - eventLogger = Some(logger) - sc.listenerBus.addListener(logger) - } - sc.listenerBus - } else { - replayerBus = Some(new SparkReplayerBus(conf)) - replayerBus.get - } - // Storage status listener must receive events first, as other listeners depend on its state - eventBus.addListener(storageStatusListener) - eventBus.addListener(storage.listener) - eventBus.addListener(jobs.listener) - eventBus.addListener(env.listener) - eventBus.addListener(exec.listener) - started = true - } - - /** - * Reconstruct a previously persisted SparkUI from logs residing in the given directory. - * - * This method must be invoked after the SparkUI has started. Return true if log files - * are found and processed. - */ - def renderFromPersistedStorage(logDir: String): Boolean = { - assume(!live, "Live Spark Web UI attempted to render from persisted storage!") - assume(started, "Spark Web UI attempted to render from persisted storage before starting!") - replayerBus.get.replay(logDir) + listenerBus.addListener(storageStatusListener) + listenerBus.addListener(storage.listener) + listenerBus.addListener(jobs.listener) + listenerBus.addListener(env.listener) + listenerBus.addListener(exec.listener) + + // Log events only if this UI is live and the feature is enabled + if (live && conf.getBoolean("spark.eventLog.enabled", false)) { + val logger = new EventLoggingListener(appName, conf) + eventLogger = Some(logger) + listenerBus.addListener(logger) + } } def stop() { diff --git a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala b/core/src/main/scala/org/apache/spark/ui/UIReloader.scala deleted file mode 100644 index f58e36a213b98..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/UIReloader.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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 - -import org.apache.spark.SparkConf - -/** - * Reload a persisted UI independently from a SparkContext. - */ -object UIReloader { - def main(args: Array[String]) { - if (args.length < 1) { - println("Usage: ./bin/spark-class org.apache.spark.ui.UIReloader [log path]") - System.exit(1) - } - - val conf = new SparkConf() - conf.set("spark.ui.port", "14040") - val ui = new SparkUI(conf, "My Application") - ui.bind() - ui.start() - val success = ui.renderFromPersistedStorage(args(0)) - if (!success) { - ui.stop() - } - - println("\nTo exit, type exit or quit.") - var line = "" - while (line != "exit" && line != "quit") { - print("> ") - line = readLine() - } - println("\nReceived signal to exit.") - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index fb42258ba7501..c2b2d41054e63 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -239,9 +239,9 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } } - override def onBlockManagerGained(blockManagerGained: SparkListenerBlockManagerGained) { + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { synchronized { - val blockManagerId = blockManagerGained.blockManagerId + val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId executorIdToBlockManagerId(executorId) = blockManagerId } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 93d5d4a24d689..409223b67737e 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -77,8 +77,7 @@ class FileLogger( } /** - * Create a new writer for the file identified by the given path. File systems currently - * supported include HDFS, S3, and the local file system. + * Create a new writer for the file identified by the given path. */ private def createWriter(): PrintWriter = { val logPath = logDir + "/" + fileIndex 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 edf3711d37899..2b51597785b98 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -57,8 +57,8 @@ private[spark] object JsonProtocol { jobEndToJson(jobEnd) case environmentUpdate: SparkListenerEnvironmentUpdate => environmentUpdateToJson(environmentUpdate) - case blockManagerGained: SparkListenerBlockManagerGained => - blockManagerGainedToJson(blockManagerGained) + case blockManagerAdded: SparkListenerBlockManagerAdded => + blockManagerAddedToJson(blockManagerAdded) case blockManagerLost: SparkListenerBlockManagerLost => blockManagerLostToJson(blockManagerLost) case unpersistRDD: SparkListenerUnpersistRDD => @@ -139,11 +139,11 @@ private[spark] object JsonProtocol { ("Classpath Entries" -> classpathEntries) } - def blockManagerGainedToJson(blockManagerGained: SparkListenerBlockManagerGained): JValue = { - val blockManagerId = blockManagerIdToJson(blockManagerGained.blockManagerId) - ("Event" -> Utils.getFormattedClassName(blockManagerGained)) ~ + def blockManagerAddedToJson(blockManagerAdded: SparkListenerBlockManagerAdded): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ ("Block Manager ID" -> blockManagerId) ~ - ("Maximum Memory" -> blockManagerGained.maxMem) + ("Maximum Memory" -> blockManagerAdded.maxMem) } def blockManagerLostToJson(blockManagerLost: SparkListenerBlockManagerLost): JValue = { @@ -370,7 +370,7 @@ private[spark] object JsonProtocol { val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) - val blockManagerGained = Utils.getFormattedClassName(SparkListenerBlockManagerGained) + val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerLost = Utils.getFormattedClassName(SparkListenerBlockManagerLost) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) val shutdown = Utils.getFormattedClassName(SparkListenerShutdown) @@ -384,7 +384,7 @@ private[spark] object JsonProtocol { case `jobStart` => jobStartFromJson(json) case `jobEnd` => jobEndFromJson(json) case `environmentUpdate` => environmentUpdateFromJson(json) - case `blockManagerGained` => blockManagerGainedFromJson(json) + case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerLost` => blockManagerLostFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) case `shutdown` => SparkListenerShutdown @@ -444,10 +444,10 @@ private[spark] object JsonProtocol { SparkListenerEnvironmentUpdate(environmentDetails) } - def blockManagerGainedFromJson(json: JValue): SparkListenerBlockManagerGained = { + def blockManagerAddedFromJson(json: JValue): SparkListenerBlockManagerAdded = { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] - SparkListenerBlockManagerGained(blockManagerId, maxMem) + SparkListenerBlockManagerAdded(blockManagerId, maxMem) } def blockManagerLostFromJson(json: JValue): SparkListenerBlockManagerLost = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 4b53107d73624..9274e01632d58 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -270,7 +270,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. val dagScheduler = new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} } val numFreeCores = 1 diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 9db59da0d1ec2..c0ca97b2c5ebd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -43,7 +43,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) taskScheduler.endedTasks(taskInfo.index) = reason } - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} override def executorLost(execId: String) {} 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 fb6eb5cd4fe37..da3933cff04cc 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -47,7 +47,7 @@ class JsonProtocolSuite extends FunSuite { "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) )) - val blockManagerGained = SparkListenerBlockManagerGained( + val blockManagerAdded = SparkListenerBlockManagerAdded( BlockManagerId("Stars", "In your multitude...", 300, 400), 500) val blockManagerLost = SparkListenerBlockManagerLost( BlockManagerId("Scarce", "to be counted...", 100, 200)) @@ -61,7 +61,7 @@ class JsonProtocolSuite extends FunSuite { testEvent(jobStart, jobStartJsonString) testEvent(jobEnd, jobEndJsonString) testEvent(environmentUpdate, environmentUpdateJsonString) - testEvent(blockManagerGained, blockManagerGainedJsonString) + testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerLost, blockManagerLostJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) testEvent(SparkListenerShutdown, shutdownJsonString) @@ -205,7 +205,7 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) - case (e1: SparkListenerBlockManagerGained, e2: SparkListenerBlockManagerGained) => + case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => assert(e1.maxMem == e2.maxMem) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerBlockManagerLost, e2: SparkListenerBlockManagerLost) => @@ -540,9 +540,9 @@ class JsonProtocolSuite extends FunSuite { "Classpath Entries":{"Super library":"/tmp/super_library"}} """ - private val blockManagerGainedJsonString = + private val blockManagerAddedJsonString = """ - {"Event":"SparkListenerBlockManagerGained","Block Manager ID":{"Executor ID":"Stars", + {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars", "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} """ diff --git a/docs/configuration.md b/docs/configuration.md index 001d01ae0d1e3..16ee5ec0f230f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -509,7 +509,7 @@ Apart from these, the following properties are also available, and may be useful spark.eventLog.dir - /tmp/spark-events + file:///tmp/spark-events Base directory in which spark events are logged, if spark.eventLog.enabled is true. Within this base directory, Spark creates a sub-directory for each application, and logs the events From f80bd31ef1446996d1be80fdae30c1a5a2eaf149 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Mar 2014 16:07:54 -0700 Subject: [PATCH 65/68] Simplify static handler and BlockManager status update logic This commit gets rid of BlockManagerStatusListener, which is there only because of initialization ordering issues. The solution is to declare LiveListenerBus from the onset, pass this into BlockManagerMasterActor, and have the LiveListenerBus buffer all events until all relevant listeners are registered. This is done by putting the creation of the asynchronous listener thread into a start() method, such that all queued events are not actually released to registered listeners until this is called and the listener thread is created. This also includes a couple of smaller clean-ups suggested by @pwendell. --- .../scala/org/apache/spark/CacheManager.scala | 4 +- .../scala/org/apache/spark/SparkContext.scala | 38 +++++++---- .../scala/org/apache/spark/SparkEnv.scala | 24 ++++--- .../apache/spark/deploy/master/Master.scala | 8 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 47 +++++++------- .../apache/spark/deploy/worker/Worker.scala | 5 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 19 +++--- .../spark/scheduler/LiveListenerBus.scala | 33 ++++++---- .../spark/scheduler/ReplayListenerBus.scala | 7 +-- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 2 +- .../spark/storage/BlockManagerMaster.scala | 10 +-- .../storage/BlockManagerMasterActor.scala | 25 ++++---- .../storage/BlockManagerStatusListener.scala | 63 ------------------- .../apache/spark/storage/ThreadingTest.scala | 7 ++- .../org/apache/spark/ui/JettyUtils.scala | 40 ++++-------- .../scala/org/apache/spark/ui/SparkUI.scala | 31 +++------ .../scala/org/apache/spark/ui/UIUtils.scala | 8 +-- .../java/org/apache/spark/JavaAPISuite.java | 23 +++---- .../org/apache/spark/CacheManagerSuite.scala | 4 +- .../org/apache/spark/PipedRDDSuite.scala | 5 +- .../spark/storage/BlockManagerSuite.scala | 4 +- .../scala/org/apache/spark/ui/UISuite.scala | 16 +++-- 23 files changed, 176 insertions(+), 249 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 5f50a5dd6e414..c7893f288b4b5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -105,9 +105,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // Update task metrics to include any blocks whose storage status is updated val metrics = context.taskMetrics - if (metrics != null) { - metrics.updatedBlocks = Some(updatedBlocks) - } + metrics.updatedBlocks = Some(updatedBlocks) returnValue diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c564760b3010b..ff8d0e9ae32ed 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -132,6 +132,9 @@ class SparkContext( if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // An asynchronous listener bus for Spark events + private[spark] val listenerBus = new LiveListenerBus + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.create( conf, @@ -139,7 +142,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, isDriver = true, - isLocal = isLocal) + isLocal = isLocal, + listenerBus = listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp @@ -195,14 +199,27 @@ class SparkContext( } executorEnvs("SPARK_USER") = sparkUser - // An asynchronous listener bus for Spark events - private[spark] val listenerBus = new LiveListenerBus - // Start the UI before posting events to listener bus, because the UI listens for Spark events private[spark] val ui = new SparkUI(this) ui.bind() ui.start() + // Optionally log SparkListenerEvents + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (conf.getBoolean("spark.eventLog.enabled", false)) { + val logger = new EventLoggingListener(appName, conf) + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // Information needed to replay logged events, if any + private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = + eventLogger.map { logger => Some(logger.info) }.getOrElse(None) + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() + // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) taskScheduler.start() @@ -211,7 +228,6 @@ class SparkContext( dagScheduler.start() postEnvironmentUpdate() - listenForBlockManagerUpdates() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -811,6 +827,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { ui.stop() + eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -1042,24 +1059,19 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() - /** Post the environment update event if the listener bus is ready */ + /** Post the environment update event once the task scheduler is ready. */ private def postEnvironmentUpdate() { - Option(listenerBus).foreach { bus => + if (taskScheduler != null) { val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq val addedFilePaths = addedFiles.keys.toSeq val environmentDetails = SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) - bus.post(environmentUpdate) + listenerBus.post(environmentUpdate) } } - /** Start listening for block manager status update events */ - private def listenForBlockManagerUpdates() { - env.blockManager.master.listener.map(_.setListenerBus(listenerBus)) - } - /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9d74b7e299f97..62ee3c08c09f1 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -29,6 +29,7 @@ import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.ConnectionManager +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{Serializer, SerializerManager} import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -123,7 +124,13 @@ object SparkEnv extends Logging { hostname: String, port: Int, isDriver: Boolean, - isLocal: Boolean): SparkEnv = { + isLocal: Boolean, + listenerBus: LiveListenerBus = null): SparkEnv = { + + // Listener bus is only used on the driver + if (isDriver) { + assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") + } val securityManager = new SecurityManager(conf) @@ -168,19 +175,10 @@ object SparkEnv extends Logging { } } - val blockManagerStatusListener = new BlockManagerStatusListener - - // Lazy because an akka actor cannot be instantiated outside of Props - lazy val blockManagerMasterActor = { - val actor = new BlockManagerMasterActor(isLocal, conf) - actor.registerListener(blockManagerStatusListener) - actor - } - val blockManagerMaster = new BlockManagerMaster( - registerOrLookup("BlockManagerMaster", blockManagerMasterActor), - conf, - blockManagerStatusListener) + registerOrLookup("BlockManagerMaster", + new BlockManagerMasterActor(isLocal, conf, listenerBus)), + conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, securityManager) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7f49ec96d5e66..d44074a69ec99 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -116,7 +116,7 @@ private[spark] class Master( // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.bind() - masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get + masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) masterMetricsSystem.registerSource(masterSource) @@ -382,7 +382,7 @@ private[spark] class Master( } case RequestWebUIPort => { - sender ! WebUIPortResponse(webUi.boundPort.getOrElse(-1)) + sender ! WebUIPortResponse(webUi.boundPort) } } @@ -622,6 +622,10 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { + appIdToUI.get(a.id).foreach { ui => + ui.stop() + webUi.detachUI(ui) + } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index e0e7cb1bd38db..193858448c557 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -18,13 +18,11 @@ package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler -import org.eclipse.jetty.server.handler.ContextHandlerCollection import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{ServerInfo, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -35,9 +33,7 @@ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - var server: Option[Server] = None - var boundPort: Option[Int] = None - var rootHandler: Option[ContextHandlerCollection] = None + var serverInfo: Option[ServerInfo] = None private val host = Utils.localHostName() private val port = requestedPort @@ -62,11 +58,8 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def bind() { try { - val (srv, bPort, handlerCollection) = startJettyServer(host, port, handlers, master.conf) - server = Some(srv) - boundPort = Some(bPort) - rootHandler = Some(handlerCollection) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) + serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Master JettyUtils", e) @@ -74,23 +67,35 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ def attachUI(ui: SparkUI) { - rootHandler.foreach { root => - // Redirect all requests for static resources - val staticHandler = createStaticRedirectHandler("/static", ui.basePath) - val handlersToRegister = ui.handlers ++ Seq(staticHandler) - for (handler <- handlersToRegister) { - root.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } + assert(serverInfo.isDefined, "Master UI must be initialized before attaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be initialized before detaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() } + rootHandler.removeHandler(handler) } } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not initialized!") + serverInfo.get.server.stop() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index afaabedffefea..74cd4d96b0964 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -124,7 +124,7 @@ private[spark] class Worker( createWorkDir() webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() + webUi.bind() registerWithMaster() metricsSystem.registerSource(workerSource) @@ -150,8 +150,7 @@ private[spark] class Worker( for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) - actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, - publicAddress) + actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort, publicAddress) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index e9faf2d01000d..75670f85b951c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,12 +19,11 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, UIUtils} +import org.apache.spark.ui.{JettyUtils, ServerInfo, UIUtils} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -39,8 +38,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val port = requestedPort.getOrElse( worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) - var server: Option[Server] = None - var boundPort: Option[Int] = None + var serverInfo: Option[ServerInfo] = None val indexPage = new IndexPage(this) @@ -58,12 +56,10 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) ) - def start() { + def bind() { try { - val (srv, bPort, _) = JettyUtils.startJettyServer(host, port, handlers, worker.conf) - server = Some(srv) - boundPort = Some(bPort) - logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) + serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Worker JettyUtils", e) @@ -71,6 +67,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } } + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -197,7 +195,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not initialized!") + serverInfo.get.server.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 71cc47eda33eb..af48466365acf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -30,21 +30,28 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false - // Create a new daemon thread to listen for events. This thread is stopped when it receives - // a SparkListenerShutdown event, using the stop method. - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - val shutdown = postToAll(event) - if (shutdown) { - // Get out of the while loop and shutdown the daemon thread - return + /** + * Create a new daemon thread to listen for events. Until this thread has started, all posted + * events are buffered. Only after this is called will the buffered events be released to all + * attached listeners. + * + * This thread is stopped when it receives a SparkListenerShutdown event, using the stop method. + */ + def start() { + new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + val shutdown = postToAll(event) + if (shutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } } } - } - }.start() + }.start() + } def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index e2963b3287b28..4e27d959d4534 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -69,7 +69,7 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus logPaths.foreach { path => // In case there is an exception, keep track of the highest level stream to close it later var streamToClose: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { val fstream = fileSystem.open(path) val bstream = new FastBufferedInputStream(fstream) @@ -85,9 +85,8 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus } } catch { case e: Exception => - logWarning("Exception in parsing Spark event log %s".format(path)) - logWarning(currentLine + "\n") - logDebug(e.getMessage + e.getStackTraceString) + logError("Exception in parsing Spark event log %s".format(path), e) + logError("Malformed line: %s\n".format(currentLine)) } finally { streamToClose.foreach(_.close()) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index b1bee7beebe7c..25b7472a99cdb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.ui.eventLogInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8d4a122f55051..690718832a3d0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -504,7 +504,7 @@ private[spark] class BlockManager( blockId: BlockId, file: File, serializer: Serializer, - bufferSize: Int) : BlockObjectWriter = { + bufferSize: Int): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 95e1ecb797440..ed6937851b836 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,15 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster( - var driverActor: ActorRef, - conf: SparkConf, - val listener: Option[BlockManagerStatusListener] = None) - extends Logging { - - def this(driverActor: ActorRef, conf: SparkConf, listener: BlockManagerStatusListener) = - this(driverActor, conf, Some(listener)) - +class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 1e8edc31ba083..7c479b17b46d9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,7 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} -import org.apache.spark.scheduler.{SparkListenerBlockManagerAdded, SparkListenerBlockManagerLost} +import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -37,7 +37,10 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { +class BlockManagerMasterActor( + val isLocal: Boolean, + conf: SparkConf, + listenerBus: LiveListenerBus) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] @@ -50,9 +53,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) - // Post block manager status updates to SparkContext through a listener - private var statusListener: Option[BlockManagerStatusListener] = None - val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong @@ -70,10 +70,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act super.preStart() } - def registerListener(listener: BlockManagerStatusListener) { - statusListener = Some(listener) - } - def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) @@ -168,7 +164,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } } val blockManagerLost = SparkListenerBlockManagerLost(blockManagerId) - statusListener.foreach(_.onBlockManagerLost(blockManagerLost)) + listenerBus.post(blockManagerLost) } private def expireDeadHosts() { @@ -246,7 +242,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } val blockManagerAdded = SparkListenerBlockManagerAdded(id, maxMemSize) - statusListener.foreach(_.onBlockManagerAdded(blockManagerAdded)) + listenerBus.post(blockManagerAdded) } private def updateBlockInfo( @@ -340,8 +336,11 @@ private[spark] class BlockManagerInfo( _lastSeenMs = System.currentTimeMillis() } - def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + def updateBlockInfo( + blockId: BlockId, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long) { updateLastSeenMs() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala deleted file mode 100644 index be3755d189ccb..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerStatusListener.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.storage - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.scheduler._ - -/** - * A listener for BlockManager status updates. - * - * This listener provides a way to post executor storage status information as soon as it - * is available (i.e. immediately after the associated BlockManager has registered with the - * driver). This is necessary because the SparkContext is only notified when an executor is - * launched, but by then the storage information is not ready yet. - * - * Further, it is possible for a BlockManager be registered before the listener bus on the - * driver is initialized (e.g. the driver's own BlockManager), in which case the corresponding - * event should be buffered. - */ -private[spark] class BlockManagerStatusListener extends SparkListener { - private var _listenerBus: Option[LiveListenerBus] = None - - // Buffer any events received before the listener bus is ready - private val bufferedEvents = new ArrayBuffer[SparkListenerEvent] - - /** - * Set the listener bus. If there are buffered events, post them all to the listener bus. - */ - def setListenerBus(listenerBus: LiveListenerBus) = { - _listenerBus = Some(listenerBus) - bufferedEvents.map(listenerBus.postToAll) - } - - /** - * Post the event if the listener bus is ready; otherwise, buffer it. - */ - private def postOrBuffer(event: SparkListenerEvent) { - _listenerBus.map(_.post(event)).getOrElse { bufferedEvents += event } - } - - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) = - postOrBuffer(blockManagerAdded) - - override def onBlockManagerLost(blockManagerLost: SparkListenerBlockManagerLost) = - postOrBuffer(blockManagerLost) - -} diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 36f2a0fd02724..226ed2a132b00 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,9 +22,9 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -97,7 +97,8 @@ private[spark] object ThreadingTest { val conf = new SparkConf() val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, new SecurityManager(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index f525a934cac6a..a1bb6cabd3c9c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -102,23 +102,13 @@ private[spark] object JettyUtils extends Logging { srcPath: String, destPath: String, basePath: String = ""): ServletContextHandler = { - val prefixedDestPath = attachPrefix(basePath, destPath) - val transformURL = (oldURL: String) => { - // Make sure we don't end up with "//" in the middle - new URL(new URL(oldURL), prefixedDestPath).toString - } - createRedirectModifyHandler(srcPath, transformURL, basePath) - } - - /** Create a handler that always redirects the user to a modified path */ - def createRedirectModifyHandler( - srcPath: String, - modifyURL: String => String, - basePath: String = ""): ServletContextHandler = { val prefixedSrcPath = attachPrefix(basePath, srcPath) + val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { - response.sendRedirect(modifyURL(request.getRequestURL.toString)) + // Make sure we don't end up with "//" in the middle + val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString + response.sendRedirect(newUrl) } } val contextHandler = new ServletContextHandler @@ -143,13 +133,6 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Create a handler that properly redirects all requests for a static directory */ - def createStaticRedirectHandler( - srcPath: String, - basePath: String = ""): ServletContextHandler = { - createRedirectModifyHandler(srcPath, getRedirectStaticURL, basePath) - } - private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { @@ -186,7 +169,7 @@ private[spark] object JettyUtils extends Logging { hostName: String, port: Int, handlers: Seq[ServletContextHandler], - conf: SparkConf): (Server, Int, ContextHandlerCollection) = { + conf: SparkConf): ServerInfo = { val collection = new ContextHandlerCollection collection.setHandlers(handlers.toArray) @@ -214,13 +197,7 @@ private[spark] object JettyUtils extends Logging { } val (server, boundPort) = connect(port) - (server, boundPort, collection) - } - - /** Return the correct URL for a static resource by removing the prefix */ - private def getRedirectStaticURL(url: String): String = { - val newPath = "/static.*".r.findFirstIn(url).mkString("") - new URL(new URL(url), newPath).toString + ServerInfo(server, boundPort, collection) } /** Attach a prefix to the given path, but avoid returning an empty path */ @@ -228,3 +205,8 @@ private[spark] object JettyUtils extends Logging { if (basePath == "") relativePath else (basePath + relativePath).stripSuffix("/") } } + +private[spark] case class ServerInfo( + server: Server, + boundPort: Int, + rootHandler: ContextHandlerCollection) 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 bc138c5398eb7..db6a996325e6e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -50,8 +50,7 @@ private[spark] class SparkUI( private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - private var boundPort: Option[Int] = None - private var server: Option[Server] = None + private var serverInfo: Option[ServerInfo] = None private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -78,20 +77,11 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - // Only log events if this SparkUI is live - private var eventLogger: Option[EventLoggingListener] = None - - // Information needed to replay the events logged by this UI, if any - lazy val eventLogInfo: Option[EventLoggingInfo] = - eventLogger.map { l => Some(l.info) }.getOrElse(None) - /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort, _) = startJettyServer(host, port, handlers, sc.conf) - logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) - server = Some(srv) - boundPort = Some(usedPort) + serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) + logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Spark JettyUtils", e) @@ -99,6 +89,8 @@ private[spark] class SparkUI( } } + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + /** Initialize all components of the server */ def start() { storage.start() @@ -112,22 +104,15 @@ private[spark] class SparkUI( listenerBus.addListener(jobs.listener) listenerBus.addListener(env.listener) listenerBus.addListener(exec.listener) - - // Log events only if this UI is live and the feature is enabled - if (live && conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf) - eventLogger = Some(logger) - listenerBus.addListener(logger) - } } def stop() { - server.foreach(_.stop()) - eventLogger.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not initialized!") + serverInfo.get.server.stop() logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") + private[spark] def appUIAddress = "http://" + host + ":" + boundPort } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index beb1d1ce386c9..a487924effbff 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -61,18 +61,18 @@ private[spark] object UIUtils { - - - + {appName} - {title}