diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java index 31d4e6fefc289..667fccccd5428 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java @@ -25,21 +25,21 @@ * Abstraction for a local key/value store for storing app data. * *

- * Use {@link KVStoreBuilder} to create an instance. There are two main features provided by the - * implementations of this interface: + * There are two main features provided by the implementations of this interface: *

* - * + *

+ * Data will be serialized to and deserialized from the underlying data store using a + * {@link KVStoreSerializer}, which can be customized by the application. The serializer is + * based on Jackson, so it supports all the Jackson annotations for controlling the serialization + * of app-defined types. + *

+ * + *

+ * Data is also automatically compressed to save disk space. + *

* *

Automatic Key Management

* @@ -78,26 +78,6 @@ public interface KVStore extends Closeable { */ void setMetadata(Object value) throws Exception; - /** - * Returns the value of a specific key, deserialized to the given type. - */ - T get(byte[] key, Class klass) throws Exception; - - /** - * Write a single key directly to the store, atomically. - */ - void put(byte[] key, Object value) throws Exception; - - /** - * Removes a key from the store. - */ - void delete(byte[] key) throws Exception; - - /** - * Returns an iterator that will only list values with keys starting with the given prefix. - */ - KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception; - /** * Read a specific instance of an object. */ diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java index a68c37942dee4..ab86dc35f14a2 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java @@ -31,9 +31,12 @@ *

* *

- * The iterators returns by this view are of type {@link KVStoreIterator}; they auto-close + * The iterators returned by this view are of type {@link KVStoreIterator}; they auto-close * when used in a for loop that exhausts their contents, but when used manually, they need - * to be closed explicitly unless all elements are read. + * to be closed explicitly unless all elements are read. For this reason, it's recommended + * that {@link #last(Object)} and {@link #max(long)} be used to make it easier to release + * resources associated with the iterator by better controlling how many elements will be + * retrieved. *

*/ public abstract class KVStoreView implements Iterable { @@ -43,7 +46,9 @@ public abstract class KVStoreView implements Iterable { boolean ascending = true; String index = KVIndex.NATURAL_INDEX_NAME; Object first = null; + Object last = null; long skip = 0L; + long max = Long.MAX_VALUE; public KVStoreView(Class type) { this.type = type; @@ -74,7 +79,25 @@ public KVStoreView first(Object value) { } /** - * Skips a number of elements in the resulting iterator. + * Stops iteration at the given value of the chosen index. + */ + public KVStoreView last(Object value) { + this.last = value; + return this; + } + + /** + * Stops iteration after a number of elements has been retrieved. + */ + public KVStoreView max(long max) { + Preconditions.checkArgument(max > 0L, "max must be positive."); + this.max = max; + return this; + } + + /** + * Skips a number of elements at the start of iteration. Skipped elements are not accounted + * when using {@link #max(long)}. */ public KVStoreView skip(long n) { this.skip = n; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 337b9541e2879..f3ab58a01af21 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -69,7 +69,7 @@ public LevelDB(File path, KVStoreSerializer serializer) throws Exception { this.types = new ConcurrentHashMap<>(); Options options = new Options(); - options.createIfMissing(!path.exists()); + options.createIfMissing(true); this._db = new AtomicReference<>(JniDBFactory.factory.open(path, options)); byte[] versionData = db().get(STORE_VERSION_KEY); @@ -109,8 +109,7 @@ public void setMetadata(Object value) throws Exception { } } - @Override - public T get(byte[] key, Class klass) throws Exception { + T get(byte[] key, Class klass) throws Exception { byte[] data = db().get(key); if (data == null) { throw new NoSuchElementException(new String(key, UTF_8)); @@ -118,22 +117,11 @@ public T get(byte[] key, Class klass) throws Exception { return serializer.deserialize(data, klass); } - @Override - public void put(byte[] key, Object value) throws Exception { + private void put(byte[] key, Object value) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); db().put(key, serializer.serialize(value)); } - @Override - public void delete(byte[] key) throws Exception { - db().delete(key); - } - - @Override - public KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception { - throw new UnsupportedOperationException(); - } - @Override public T read(Class klass, Object naturalKey) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); @@ -189,6 +177,8 @@ public void delete(Class type, Object naturalKey, boolean sync) throws Except batch.write(sync); } } + } catch (NoSuchElementException nse) { + // Ignore. } finally { batch.close(); } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index 3b00c171740db..10cb580d85e0c 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -38,10 +38,12 @@ class LevelDBIterator implements KVStoreIterator { private final LevelDBTypeInfo.Index index; private final byte[] indexKeyPrefix; private final byte[] end; + private final long max; private boolean checkedNext; private T next; private boolean closed; + private long count; /** * Creates a simple iterator over db keys. @@ -55,6 +57,7 @@ class LevelDBIterator implements KVStoreIterator { this.it = db.db().iterator(); this.indexKeyPrefix = keyPrefix; this.end = null; + this.max = -1L; it.seek(keyPrefix); } @@ -69,6 +72,7 @@ class LevelDBIterator implements KVStoreIterator { this.ti = db.getTypeInfo(type); this.index = ti.index(params.index); this.indexKeyPrefix = index.keyPrefix(); + this.max = params.max; byte[] firstKey; if (params.first != null) { @@ -84,14 +88,27 @@ class LevelDBIterator implements KVStoreIterator { } it.seek(firstKey); + byte[] end = null; if (ascending) { - this.end = index.end(); + end = params.last != null ? index.end(params.last) : index.end(); } else { - this.end = null; + if (params.last != null) { + end = index.start(params.last); + } if (it.hasNext()) { - it.next(); + // When descending, the caller may have set up the start of iteration at a non-existant + // entry that is guaranteed to be after the desired entry. For example, if you have a + // compound key (a, b) where b is a, integer, you may seek to the end of the elements that + // have the same "a" value by specifying Integer.MAX_VALUE for "b", and that value may not + // exist in the database. So need to check here whether the next value actually belongs to + // the set being returned by the iterator before advancing. + byte[] nextKey = it.peekNext().getKey(); + if (compare(nextKey, indexKeyPrefix) <= 0) { + it.next(); + } } } + this.end = end; if (params.skip > 0) { skip(params.skip); @@ -164,7 +181,23 @@ public synchronized void close() throws IOException { } } + /** + * Because it's tricky to expose closeable iterators through many internal APIs, especially + * when Scala wrappers are used, this makes sure that, hopefully, the JNI resources held by + * the iterator will eventually be released. + */ + @Override + protected void finalize() throws Throwable { + if (db.db() != null) { + close(); + } + } + private T loadNext() { + if (count >= max) { + return null; + } + try { while (true) { boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); @@ -191,11 +224,16 @@ private T loadNext() { return null; } - // If there's a known end key and it's found, stop. - if (end != null && Arrays.equals(nextKey, end)) { - return null; + // If there's a known end key and iteration has gone past it, stop. + if (end != null) { + int comp = compare(nextKey, end) * (ascending ? 1 : -1); + if (comp > 0) { + return null; + } } + count++; + // Next element is part of the iteration, return it. if (index == null || index.isCopy()) { return db.serializer.deserialize(nextEntry.getValue(), type); @@ -246,4 +284,17 @@ private byte[] stitch(byte[]... comps) { return dest; } + private int compare(byte[] a, byte[] b) { + int diff = 0; + int minLen = Math.min(a.length, b.length); + for (int i = 0; i < minLen; i++) { + diff += (a[i] - b[i]); + if (diff != 0) { + return diff; + } + } + + return a.length - b.length; + } + } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 88c7cc08984bb..0119d58d8ae26 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -25,6 +25,7 @@ import java.util.Iterator; import java.util.List; import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; @@ -152,109 +153,170 @@ public static void cleanup() throws Exception { @Test public void naturalIndex() throws Exception { - testIteration(NATURAL_ORDER, view(), null); + testIteration(NATURAL_ORDER, view(), null, null); } @Test public void refIndex() throws Exception { - testIteration(REF_INDEX_ORDER, view().index("id"), null); + testIteration(REF_INDEX_ORDER, view().index("id"), null, null); } @Test public void copyIndex() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name"), null); + testIteration(COPY_INDEX_ORDER, view().index("name"), null, null); } @Test public void numericIndex() throws Exception { - testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null); + testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null, null); } @Test public void naturalIndexDescending() throws Exception { - testIteration(NATURAL_ORDER, view().reverse(), null); + testIteration(NATURAL_ORDER, view().reverse(), null, null); } @Test public void refIndexDescending() throws Exception { - testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null); + testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null, null); } @Test public void copyIndexDescending() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null); + testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null, null); } @Test public void numericIndexDescending() throws Exception { - testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null, null); } @Test public void naturalIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NATURAL_ORDER, view().first(first.key), first); + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().first(first.key), first, null); } @Test public void refIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first); + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first, null); } @Test public void copyIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first); + CustomType1 first = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first, null); } @Test public void numericIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first); + CustomType1 first = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first, null); } @Test public void naturalIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NATURAL_ORDER, view().reverse().first(first.key), first); + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().first(first.key), first, null); } @Test public void refIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first); + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first, null); } @Test public void copyIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); + CustomType1 first = pickLimit(); testIteration(COPY_INDEX_ORDER, view().reverse().index("name").first(first.name), - first); + first, null); } @Test public void numericIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); + CustomType1 first = pickLimit(); testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), - first); + first, null); } @Test public void naturalIndexWithSkip() throws Exception { - testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null); + testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null, null); } @Test public void refIndexWithSkip() throws Exception { testIteration(REF_INDEX_ORDER, view().index("id").skip(RND.nextInt(allEntries.size() / 2)), - null); + null, null); } @Test public void copyIndexWithSkip() throws Exception { testIteration(COPY_INDEX_ORDER, view().index("name").skip(RND.nextInt(allEntries.size() / 2)), - null); + null, null); + } + + @Test + public void naturalIndexWithMax() throws Exception { + testIteration(NATURAL_ORDER, view().max(RND.nextInt(allEntries.size() / 2)), null, null); + } + + @Test + public void copyIndexWithMax() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").max(RND.nextInt(allEntries.size() / 2)), + null, null); + } + + @Test + public void naturalIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().last(last.key), null, last); + } + + @Test + public void refIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").last(last.name), null, last); + } + + @Test + public void numericIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").last(last.num), null, last); + } + + @Test + public void naturalIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().last(last.key), null, last); + } + + @Test + public void refIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").last(last.name), + null, last); + } + + @Test + public void numericIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").last(last.num), + null, last); } @Test @@ -272,8 +334,8 @@ public void testRefWithIntNaturalKey() throws Exception { } } - private CustomType1 pickFirst() { - // Picks a first element that has clashes with other elements in the given index. + private CustomType1 pickLimit() { + // Picks an element that has clashes with other elements in the given index. return clashingEntries.get(RND.nextInt(clashingEntries.size())); } @@ -297,22 +359,32 @@ private > int compareWithFallback( private void testIteration( final BaseComparator order, final KVStoreView params, - final CustomType1 first) throws Exception { + final CustomType1 first, + final CustomType1 last) throws Exception { List indexOrder = sortBy(order.fallback()); if (!params.ascending) { indexOrder = Lists.reverse(indexOrder); } Iterable expected = indexOrder; + BaseComparator expectedOrder = params.ascending ? order : order.reverse(); + if (first != null) { - final BaseComparator expectedOrder = params.ascending ? order : order.reverse(); expected = Iterables.filter(expected, v -> expectedOrder.compare(first, v) <= 0); } + if (last != null) { + expected = Iterables.filter(expected, v -> expectedOrder.compare(v, last) <= 0); + } + if (params.skip > 0) { expected = Iterables.skip(expected, (int) params.skip); } + if (params.max != Long.MAX_VALUE) { + expected = Iterables.limit(expected, (int) params.max); + } + List actual = collect(params); compareLists(expected, actual); } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index c3baf76589286..1f88aae0be2aa 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -74,29 +74,6 @@ public void testReopenAndVersionCheckDb() throws Exception { } } - @Test - public void testStringWriteReadDelete() throws Exception { - String string = "testString"; - byte[] key = string.getBytes(UTF_8); - testReadWriteDelete(key, string); - } - - @Test - public void testIntWriteReadDelete() throws Exception { - int value = 42; - byte[] key = "key".getBytes(UTF_8); - testReadWriteDelete(key, value); - } - - @Test - public void testSimpleTypeWriteReadDelete() throws Exception { - byte[] key = "testKey".getBytes(UTF_8); - CustomType1 t = new CustomType1(); - t.id = "id"; - t.name = "name"; - testReadWriteDelete(key, t); - } - @Test public void testObjectWriteReadDelete() throws Exception { CustomType1 t = new CustomType1(); @@ -268,26 +245,6 @@ private int countKeys(Class type) throws Exception { return count; } - private void testReadWriteDelete(byte[] key, T value) throws Exception { - try { - db.get(key, value.getClass()); - fail("Expected exception for non-existent key."); - } catch (NoSuchElementException nsee) { - // Expected. - } - - db.put(key, value); - assertEquals(value, db.get(key, value.getClass())); - - db.delete(key); - try { - db.get(key, value.getClass()); - fail("Expected exception for deleted key."); - } catch (NoSuchElementException nsee) { - // Expected. - } - } - public static class IntKeyType { @KVIndex diff --git a/core/pom.xml b/core/pom.xml index 7f245b5b6384a..a98f5da2ff407 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -67,6 +67,11 @@ spark-launcher_${scala.binary.version} ${project.version} + + org.apache.spark + spark-kvstore_${scala.binary.version} + ${project.version} + org.apache.spark spark-network-common_${scala.binary.version} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java index 9dbb565aab707..40b5f627369d5 100644 --- a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java +++ b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java @@ -23,7 +23,8 @@ public enum StageStatus { ACTIVE, COMPLETE, FAILED, - PENDING; + PENDING, + SKIPPED; public static StageStatus fromString(String str) { return EnumUtil.parseIgnoreCase(StageStatus.class, str); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 1f89306403cd5..baa9363cf0902 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -188,7 +188,7 @@ $(document).ready(function() { } $(selector).DataTable(conf); - $('#hisotry-summary [data-toggle="tooltip"]').tooltip(); + $('#history-summary [data-toggle="tooltip"]').tooltip(); }); }); }); diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7dbceb9c5c1a3..ed461f4c30496 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -54,10 +54,10 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.local.LocalSchedulerBackend +import org.apache.spark.status.AppStateStore import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} -import org.apache.spark.ui.jobs.JobProgressListener import org.apache.spark.util._ /** @@ -196,7 +196,6 @@ class SparkContext(config: SparkConf) extends Logging { private var _eventLogDir: Option[URI] = None private var _eventLogCodec: Option[String] = None private var _env: SparkEnv = _ - private var _jobProgressListener: JobProgressListener = _ private var _statusTracker: SparkStatusTracker = _ private var _progressBar: Option[ConsoleProgressBar] = None private var _ui: Option[SparkUI] = None @@ -215,6 +214,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _jars: Seq[String] = _ private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ + private var _stateStore: AppStateStore = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -268,8 +268,6 @@ class SparkContext(config: SparkConf) extends Logging { val map: ConcurrentMap[Int, RDD[_]] = new MapMaker().weakValues().makeMap[Int, RDD[_]]() map.asScala } - private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener - def statusTracker: SparkStatusTracker = _statusTracker private[spark] def progressBar: Option[ConsoleProgressBar] = _progressBar @@ -423,10 +421,9 @@ class SparkContext(config: SparkConf) extends Logging { if (master == "yarn" && deployMode == "client") System.setProperty("SPARK_YARN_MODE", "true") - // "_jobProgressListener" should be set up before creating SparkEnv because when creating - // "SparkEnv", some messages will be posted to "listenerBus" and we should not miss them. - _jobProgressListener = new JobProgressListener(_conf) - listenerBus.addListener(jobProgressListener) + // Initialize the app state store and listener before SparkEnv is created so that it gets + // all events. + _stateStore = AppStateStore.createTempStore(conf, listenerBus) // Create the Spark execution environment (cache, map output tracker, etc) _env = createSparkEnv(_conf, isLocal, listenerBus) @@ -438,7 +435,7 @@ class SparkContext(config: SparkConf) extends Logging { _conf.set("spark.repl.class.uri", replUri) } - _statusTracker = new SparkStatusTracker(this) + _statusTracker = new SparkStatusTracker(this, _stateStore) _progressBar = if (_conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { @@ -449,8 +446,8 @@ class SparkContext(config: SparkConf) extends Logging { _ui = if (conf.getBoolean("spark.ui.enabled", true)) { - Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, - _env.securityManager, appName, startTime = startTime)) + Some(SparkUI.create(Some(this), _stateStore, _conf, _env.securityManager, + appName, "", startTime)) } else { // For tests, do not enable the UI None @@ -1939,6 +1936,9 @@ class SparkContext(config: SparkConf) extends Logging { } SparkEnv.set(null) } + if (_stateStore != null) { + _stateStore.close() + } // Clear this `InheritableThreadLocal`, or it will still be inherited in child threads even this // `SparkContext` is stopped. localProperties.remove() diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index 22a553e68439a..72d96bedd50f5 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -17,7 +17,11 @@ package org.apache.spark +import java.util.Arrays + import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1.StageStatus /** * Low-level status reporting APIs for monitoring job and stage progress. @@ -33,9 +37,7 @@ import org.apache.spark.scheduler.TaskSchedulerImpl * * NOTE: this class's constructor should be considered private and may be subject to change. */ -class SparkStatusTracker private[spark] (sc: SparkContext) { - - private val jobProgressListener = sc.jobProgressListener +class SparkStatusTracker private[spark] (sc: SparkContext, store: AppStateStore) { /** * Return a list of all known jobs in a particular job group. If `jobGroup` is `null`, then @@ -46,9 +48,8 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { * its result. */ def getJobIdsForGroup(jobGroup: String): Array[Int] = { - jobProgressListener.synchronized { - jobProgressListener.jobGroupToJobIds.getOrElse(jobGroup, Seq.empty).toArray - } + val expected = Option(jobGroup) + store.jobsList(null).filter(_.jobGroup == expected).map(_.jobId).toArray } /** @@ -57,9 +58,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { * This method does not guarantee the order of the elements in its result. */ def getActiveStageIds(): Array[Int] = { - jobProgressListener.synchronized { - jobProgressListener.activeStages.values.map(_.stageId).toArray - } + store.stageList(Arrays.asList(StageStatus.ACTIVE)).map(_.stageId).toArray } /** @@ -68,19 +67,18 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { * This method does not guarantee the order of the elements in its result. */ def getActiveJobIds(): Array[Int] = { - jobProgressListener.synchronized { - jobProgressListener.activeJobs.values.map(_.jobId).toArray - } + store.jobsList(Arrays.asList(JobExecutionStatus.RUNNING)).map(_.jobId).toArray } /** * Returns job information, or `None` if the job info could not be found or was garbage collected. */ def getJobInfo(jobId: Int): Option[SparkJobInfo] = { - jobProgressListener.synchronized { - jobProgressListener.jobIdToData.get(jobId).map { data => - new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status) - } + try { + val job = store.job(jobId) + Some(new SparkJobInfoImpl(jobId, job.stageIds.toArray, job.status)) + } catch { + case _: NoSuchElementException => None } } @@ -89,21 +87,19 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { * garbage collected. */ def getStageInfo(stageId: Int): Option[SparkStageInfo] = { - jobProgressListener.synchronized { - for ( - info <- jobProgressListener.stageIdToInfo.get(stageId); - data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId)) - ) yield { - new SparkStageInfoImpl( - stageId, - info.attemptId, - info.submissionTime.getOrElse(0), - info.name, - info.numTasks, - data.numActiveTasks, - data.numCompleteTasks, - data.numFailedTasks) - } + try { + val info = store.lastStageAttempt(stageId) + Some(new SparkStageInfoImpl( + stageId, + info.attemptId, + info.submissionTime.map(_.getTime()).getOrElse(0L), + info.name, + info.numTasks, + info.numActiveTasks, + info.numCompleteTasks, + info.numFailedTasks)) + } catch { + case _: NoSuchElementException => None } } @@ -111,17 +107,16 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { * Returns information of all known executors, including host, port, cacheSize, numRunningTasks. */ def getExecutorInfos: Array[SparkExecutorInfo] = { - val executorIdToRunningTasks: Map[String, Int] = - sc.taskScheduler.asInstanceOf[TaskSchedulerImpl].runningTasksByExecutors - - sc.getExecutorStorageStatus.map { status => - val bmId = status.blockManagerId + store.executorList(true).map { exec => + val (host, port) = exec.hostPort.split(":", 2) match { + case Array(h, p) => (h, p.toInt) + case Array(h) => (h, -1) + } new SparkExecutorInfoImpl( - bmId.host, - bmId.port, - status.cacheSize, - executorIdToRunningTasks.getOrElse(bmId.executorId, 0) - ) - } + host, + port, + exec.maxMemory, + exec.activeTasks) + }.toArray } } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 3f912dc191515..531ea5cdd82e8 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import java.security.SecureRandom import java.security.cert.X509Certificate import java.util.Arrays -import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import javax.net.ssl._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} @@ -232,6 +232,30 @@ private[spark] object TestUtils { } } + /** + * Wait until at least `numExecutors` executors are up, or throw `TimeoutException` if the waiting + * time elapsed before `numExecutors` executors up. Exposed for testing. + * + * @param numExecutors the number of executors to wait at least + * @param timeout time to wait in milliseconds + */ + private[spark] def waitUntilExecutorsUp( + sc: SparkContext, + numExecutors: Int, + timeout: Long): Unit = { + val finishTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeout) + while (System.nanoTime() < finishTime) { + if (sc.statusTracker.getExecutorInfos.length > numExecutors) { + return + } + // 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) + } + throw new TimeoutException( + s"Can't find $numExecutors executors before $timeout milliseconds elapsed") + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index a370526c46f3d..7ef923170d33a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -26,6 +26,7 @@ import scala.util.control.NonFatal import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} +import com.google.common.util.concurrent.UncheckedExecutionException import org.eclipse.jetty.servlet.FilterHolder import org.apache.spark.internal.Logging @@ -40,11 +41,6 @@ import org.apache.spark.util.Clock * Incompleted applications have their update time checked on every * retrieval; if the cached entry is out of date, it is refreshed. * - * @note there must be only one instance of [[ApplicationCache]] in a - * JVM at a time. This is because a static field in [[ApplicationCacheCheckFilterRelay]] - * keeps a reference to the cache so that HTTP requests on the attempt-specific web UIs - * can probe the current cache to see if the attempts have changed. - * * Creating multiple instances will break this routing. * @param operations implementation of record access operations * @param retainedApplications number of retained applications @@ -80,7 +76,7 @@ private[history] class ApplicationCache( metrics.evictionCount.inc() val key = rm.getKey logDebug(s"Evicting entry ${key}") - operations.detachSparkUI(key.appId, key.attemptId, rm.getValue().ui) + operations.detachSparkUI(key.appId, key.attemptId, rm.getValue().loadedUI.ui) } } @@ -89,7 +85,7 @@ private[history] class ApplicationCache( * * Tagged as `protected` so as to allow subclasses in tests to access it directly */ - protected val appCache: LoadingCache[CacheKey, CacheEntry] = { + private val appCache: LoadingCache[CacheKey, CacheEntry] = { CacheBuilder.newBuilder() .maximumSize(retainedApplications) .removalListener(removalListener) @@ -101,130 +97,38 @@ private[history] class ApplicationCache( */ val metrics = new CacheMetrics("history.cache") - init() - - /** - * Perform any startup operations. - * - * This includes declaring this instance as the cache to use in the - * [[ApplicationCacheCheckFilterRelay]]. - */ - private def init(): Unit = { - ApplicationCacheCheckFilterRelay.setApplicationCache(this) - } - - /** - * Stop the cache. - * This will reset the relay in [[ApplicationCacheCheckFilterRelay]]. - */ - def stop(): Unit = { - ApplicationCacheCheckFilterRelay.resetApplicationCache() - } - - /** - * Get an entry. - * - * Cache fetch/refresh will have taken place by the time this method returns. - * @param appAndAttempt application to look up in the format needed by the history server web UI, - * `appId/attemptId` or `appId`. - * @return the entry - */ - def get(appAndAttempt: String): SparkUI = { - val parts = splitAppAndAttemptKey(appAndAttempt) - get(parts._1, parts._2) - } - - /** - * Get the Spark UI, converting a lookup failure from an exception to `None`. - * @param appAndAttempt application to look up in the format needed by the history server web UI, - * `appId/attemptId` or `appId`. - * @return the entry - */ - def getSparkUI(appAndAttempt: String): Option[SparkUI] = { + def get(appId: String, attemptId: Option[String] = None): CacheEntry = { try { - val ui = get(appAndAttempt) - Some(ui) + appCache.get(new CacheKey(appId, attemptId)) } catch { - case NonFatal(e) => e.getCause() match { - case nsee: NoSuchElementException => - None - case cause: Exception => throw cause - } + case e: UncheckedExecutionException => + throw Option(e.getCause()).getOrElse(e) } } - /** - * Get the associated spark UI. - * - * Cache fetch/refresh will have taken place by the time this method returns. - * @param appId application ID - * @param attemptId optional attempt ID - * @return the entry - */ - def get(appId: String, attemptId: Option[String]): SparkUI = { - lookupAndUpdate(appId, attemptId)._1.ui - } - - /** - * Look up the entry; update it if needed. - * @param appId application ID - * @param attemptId optional attempt ID - * @return the underlying cache entry -which can have its timestamp changed, and a flag to - * indicate that the entry has changed - */ - private def lookupAndUpdate(appId: String, attemptId: Option[String]): (CacheEntry, Boolean) = { - metrics.lookupCount.inc() - val cacheKey = CacheKey(appId, attemptId) - var entry = appCache.getIfPresent(cacheKey) - var updated = false - if (entry == null) { - // no entry, so fetch without any post-fetch probes for out-of-dateness - // this will trigger a callback to loadApplicationEntry() - entry = appCache.get(cacheKey) - } else if (!entry.completed) { - val now = clock.getTimeMillis() - log.debug(s"Probing at time $now for updated application $cacheKey -> $entry") - metrics.updateProbeCount.inc() - updated = time(metrics.updateProbeTimer) { - entry.updateProbe() - } - if (updated) { - logDebug(s"refreshing $cacheKey") - metrics.updateTriggeredCount.inc() - appCache.refresh(cacheKey) - // and repeat the lookup - entry = appCache.get(cacheKey) - } else { - // update the probe timestamp to the current time - entry.probeTime = now + def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T = { + var entry = get(appId, attemptId) + + // If the entry exists, we need to make sure we run the closure with a valid entry. So + // we need to re-try until we can lock a valid entry for read. + entry.loadedUI.lock.readLock().lock() + while (!entry.loadedUI.valid) { + entry.loadedUI.lock.readLock().unlock() + appCache.invalidate(new CacheKey(appId, attemptId)) + entry = get(appId, attemptId) + if (entry == null) { + metrics.lookupFailureCount.inc() + throw new NoSuchElementException() } + metrics.loadCount.inc() + entry.loadedUI.lock.readLock().lock() } - (entry, updated) - } - - /** - * This method is visible for testing. - * - * It looks up the cached entry *and returns a clone of it*. - * This ensures that the cached entries never leak - * @param appId application ID - * @param attemptId optional attempt ID - * @return a new entry with shared SparkUI, but copies of the other fields. - */ - def lookupCacheEntry(appId: String, attemptId: Option[String]): CacheEntry = { - val entry = lookupAndUpdate(appId, attemptId)._1 - new CacheEntry(entry.ui, entry.completed, entry.updateProbe, entry.probeTime) - } - /** - * Probe for an application being updated. - * @param appId application ID - * @param attemptId attempt ID - * @return true if an update has been triggered - */ - def checkForUpdates(appId: String, attemptId: Option[String]): Boolean = { - val (entry, updated) = lookupAndUpdate(appId, attemptId) - updated + try { + fn(entry.loadedUI.ui) + } finally { + entry.loadedUI.lock.readLock().unlock() + } } /** @@ -272,25 +176,21 @@ private[history] class ApplicationCache( * @throws NoSuchElementException if there is no matching element */ @throws[NoSuchElementException] - def loadApplicationEntry(appId: String, attemptId: Option[String]): CacheEntry = { - + private def loadApplicationEntry(appId: String, attemptId: Option[String]): CacheEntry = { logDebug(s"Loading application Entry $appId/$attemptId") metrics.loadCount.inc() time(metrics.loadTimer) { + metrics.lookupCount.inc() operations.getAppUI(appId, attemptId) match { - case Some(LoadedAppUI(ui, updateState)) => - val completed = ui.getApplicationInfoList.exists(_.attempts.last.completed) - if (completed) { - // completed spark UIs are attached directly - operations.attachSparkUI(appId, attemptId, ui, completed) - } else { + case Some(loadedUI) => + val completed = loadedUI.ui.getApplicationInfoList.exists(_.attempts.last.completed) + if (!completed) { // incomplete UIs have the cache-check filter put in front of them. - ApplicationCacheCheckFilterRelay.registerFilter(ui, appId, attemptId) - operations.attachSparkUI(appId, attemptId, ui, completed) + registerFilter(new CacheKey(appId, attemptId), loadedUI, this) } + operations.attachSparkUI(appId, attemptId, loadedUI.ui, completed) // build the cache entry - val now = clock.getTimeMillis() - val entry = new CacheEntry(ui, completed, updateState, now) + val entry = new CacheEntry(loadedUI, completed) logDebug(s"Loaded application $appId/$attemptId -> $entry") entry case None => @@ -303,32 +203,6 @@ private[history] class ApplicationCache( } } - /** - * Split up an `applicationId/attemptId` or `applicationId` key into the separate pieces. - * - * @param appAndAttempt combined key - * @return a tuple of the application ID and, if present, the attemptID - */ - def splitAppAndAttemptKey(appAndAttempt: String): (String, Option[String]) = { - val parts = appAndAttempt.split("/") - require(parts.length == 1 || parts.length == 2, s"Invalid app key $appAndAttempt") - val appId = parts(0) - val attemptId = if (parts.length > 1) Some(parts(1)) else None - (appId, attemptId) - } - - /** - * Merge an appId and optional attempt Id into a key of the form `applicationId/attemptId`. - * - * If there is an `attemptId`; `applicationId` if not. - * @param appId application ID - * @param attemptId optional attempt ID - * @return a unified string - */ - def mergeAppAndAttemptToKey(appId: String, attemptId: Option[String]): String = { - appId + attemptId.map { id => s"/$id" }.getOrElse("") - } - /** * String operator dumps the cache entries and metrics. * @return a string value, primarily for testing and diagnostics @@ -347,6 +221,26 @@ private[history] class ApplicationCache( sb.append("----\n") sb.toString() } + + /** + * Register a filter for the web UI which checks for updates to the given app/attempt + * @param ui Spark UI to attach filters to + * @param appId application ID + * @param attemptId attempt ID + */ + def registerFilter(key: CacheKey, loadedUI: LoadedAppUI, cache: ApplicationCache): Unit = { + require(loadedUI != null) + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.REQUEST) + val filter = new ApplicationCacheCheckFilter(key, loadedUI, cache) + val holder = new FilterHolder(filter) + require(loadedUI.ui.getHandlers != null, "null handlers") + loadedUI.ui.getHandlers.foreach { handler => + handler.addFilter(holder, "/*", enumDispatcher) + } + } + + def invalidate(key: CacheKey): Unit = appCache.invalidate(key) + } /** @@ -360,14 +254,12 @@ private[history] class ApplicationCache( * @param probeTime Times in milliseconds when the probe was last executed. */ private[history] final class CacheEntry( - val ui: SparkUI, - val completed: Boolean, - val updateProbe: () => Boolean, - var probeTime: Long) { + val loadedUI: LoadedAppUI, + val completed: Boolean) { /** string value is for test assertions */ override def toString: String = { - s"UI $ui, completed=$completed, probeTime=$probeTime" + s"UI ${loadedUI.ui}, completed=$completed" } } @@ -396,23 +288,17 @@ private[history] class CacheMetrics(prefix: String) extends Source { val evictionCount = new Counter() val loadCount = new Counter() val loadTimer = new Timer() - val updateProbeCount = new Counter() - val updateProbeTimer = new Timer() - val updateTriggeredCount = new Counter() /** all the counters: for registration and string conversion. */ private val counters = Seq( ("lookup.count", lookupCount), ("lookup.failure.count", lookupFailureCount), ("eviction.count", evictionCount), - ("load.count", loadCount), - ("update.probe.count", updateProbeCount), - ("update.triggered.count", updateTriggeredCount)) + ("load.count", loadCount)) /** all metrics, including timers */ private val allMetrics = counters ++ Seq( - ("load.timer", loadTimer), - ("update.probe.timer", updateProbeTimer)) + ("load.timer", loadTimer)) /** * Name of metric source @@ -498,23 +384,11 @@ private[history] trait ApplicationCacheOperations { * Implementation note: there's some abuse of a shared global entry here because * the configuration data passed to the servlet is just a string:string map. */ -private[history] class ApplicationCacheCheckFilter() extends Filter with Logging { - - import ApplicationCacheCheckFilterRelay._ - var appId: String = _ - var attemptId: Option[String] = _ - - /** - * Bind the app and attempt ID, throwing an exception if no application ID was provided. - * @param filterConfig configuration - */ - override def init(filterConfig: FilterConfig): Unit = { - - appId = Option(filterConfig.getInitParameter(APP_ID)) - .getOrElse(throw new ServletException(s"Missing Parameter $APP_ID")) - attemptId = Option(filterConfig.getInitParameter(ATTEMPT_ID)) - logDebug(s"initializing filter $this") - } +private[history] class ApplicationCacheCheckFilter( + key: CacheKey, + loadedUI: LoadedAppUI, + cache: ApplicationCache) + extends Filter with Logging { /** * Filter the request. @@ -543,123 +417,23 @@ private[history] class ApplicationCacheCheckFilter() extends Filter with Logging // if the request is for an attempt, check to see if it is in need of delete/refresh // and have the cache update the UI if so - if (operation=="HEAD" || operation=="GET" - && checkForUpdates(requestURI, appId, attemptId)) { - // send a redirect back to the same location. This will be routed - // to the *new* UI - logInfo(s"Application Attempt $appId/$attemptId updated; refreshing") - val queryStr = Option(httpRequest.getQueryString).map("?" + _).getOrElse("") - val redirectUrl = httpResponse.encodeRedirectURL(requestURI + queryStr) - httpResponse.sendRedirect(redirectUrl) - } else { - chain.doFilter(request, response) + loadedUI.lock.readLock().lock() + try { + if (loadedUI.valid) { + chain.doFilter(request, response) + } else { + cache.invalidate(key) + val queryStr = Option(httpRequest.getQueryString).map("?" + _).getOrElse("") + val redirectUrl = httpResponse.encodeRedirectURL(requestURI + queryStr) + httpResponse.sendRedirect(redirectUrl) + } + } finally { + loadedUI.lock.readLock().unlock() } } - override def destroy(): Unit = { - } - - override def toString: String = s"ApplicationCacheCheckFilter for $appId/$attemptId" -} + override def init(config: FilterConfig): Unit = { } -/** - * Global state for the [[ApplicationCacheCheckFilter]] instances, so that they can relay cache - * probes to the cache. - * - * This is an ugly workaround for the limitation of servlets and filters in the Java servlet - * API; they are still configured on the model of a list of classnames and configuration - * strings in a `web.xml` field, rather than a chain of instances wired up by hand or - * via an injection framework. There is no way to directly configure a servlet filter instance - * with a reference to the application cache which is must use: some global state is needed. - * - * Here, [[ApplicationCacheCheckFilter]] is that global state; it relays all requests - * to the singleton [[ApplicationCache]] - * - * The field `applicationCache` must be set for the filters to work - - * this is done during the construction of [[ApplicationCache]], which requires that there - * is only one cache serving requests through the WebUI. - * - * *Important* In test runs, if there is more than one [[ApplicationCache]], the relay logic - * will break: filters may not find instances. Tests must not do that. - * - */ -private[history] object ApplicationCacheCheckFilterRelay extends Logging { - // name of the app ID entry in the filter configuration. Mandatory. - val APP_ID = "appId" + override def destroy(): Unit = { } - // name of the attempt ID entry in the filter configuration. Optional. - val ATTEMPT_ID = "attemptId" - - // name of the filter to register - val FILTER_NAME = "org.apache.spark.deploy.history.ApplicationCacheCheckFilter" - - /** the application cache to relay requests to */ - @volatile - private var applicationCache: Option[ApplicationCache] = None - - /** - * Set the application cache. Logs a warning if it is overwriting an existing value - * @param cache new cache - */ - def setApplicationCache(cache: ApplicationCache): Unit = { - applicationCache.foreach( c => logWarning(s"Overwriting application cache $c")) - applicationCache = Some(cache) - } - - /** - * Reset the application cache - */ - def resetApplicationCache(): Unit = { - applicationCache = None - } - - /** - * Check to see if there has been an update - * @param requestURI URI the request came in on - * @param appId application ID - * @param attemptId attempt ID - * @return true if an update was loaded for the app/attempt - */ - def checkForUpdates(requestURI: String, appId: String, attemptId: Option[String]): Boolean = { - - logDebug(s"Checking $appId/$attemptId from $requestURI") - applicationCache match { - case Some(cache) => - try { - cache.checkForUpdates(appId, attemptId) - } catch { - case ex: Exception => - // something went wrong. Keep going with the existing UI - logWarning(s"When checking for $appId/$attemptId from $requestURI", ex) - false - } - - case None => - logWarning("No application cache instance defined") - false - } - } - - - /** - * Register a filter for the web UI which checks for updates to the given app/attempt - * @param ui Spark UI to attach filters to - * @param appId application ID - * @param attemptId attempt ID - */ - def registerFilter( - ui: SparkUI, - appId: String, - attemptId: Option[String] ): Unit = { - require(ui != null) - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.REQUEST) - val holder = new FilterHolder() - holder.setClassName(FILTER_NAME) - holder.setInitParameter(APP_ID, appId) - attemptId.foreach( id => holder.setInitParameter(ATTEMPT_ID, id)) - require(ui.getHandlers != null, "null handlers") - ui.getHandlers.foreach { handler => - handler.addFilter(holder, "/*", enumDispatcher) - } - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 6d8758a3d3b1d..966106a9f320e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.history +import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.zip.ZipOutputStream import scala.xml.Node @@ -46,33 +47,40 @@ private[spark] case class ApplicationHistoryInfo( } } -/** - * A probe which can be invoked to see if a loaded Web UI has been updated. - * The probe is expected to be relative purely to that of the UI returned - * in the same [[LoadedAppUI]] instance. That is, whenever a new UI is loaded, - * the probe returned with it is the one that must be used to check for it - * being out of date; previous probes must be discarded. - */ -private[history] abstract class HistoryUpdateProbe { - /** - * Return true if the history provider has a later version of the application - * attempt than the one against this probe was constructed. - * @return - */ - def isUpdated(): Boolean -} - /** * All the information returned from a call to `getAppUI()`: the new UI * and any required update state. * @param ui Spark UI * @param updateProbe probe to call to check on the update state of this application attempt */ -private[history] case class LoadedAppUI( - ui: SparkUI, - updateProbe: () => Boolean) +private[spark] case class LoadedAppUI(ui: SparkUI) { -private[history] abstract class ApplicationHistoryProvider { + val lock = new ReentrantReadWriteLock() + + @volatile private var _valid = true + + def valid: Boolean = _valid + + def invalidate(): Unit = { + lock.writeLock().lock() + try { + _valid = false + } finally { + lock.writeLock().unlock() + } + } + +} + +private[spark] abstract class ApplicationHistoryProvider { + + /** + * The number of applications available for listing. Separate method in case it's cheaper + * to get a count than to calculate the whole listing. + * + * @return The number of available applications. + */ + def getAppCount(): Int = getListing().size /** * Returns the count of application event logs that the provider is currently still processing. @@ -144,4 +152,10 @@ private[history] abstract class ApplicationHistoryProvider { * @return html text to display when the application list is empty */ def getEmptyListingHtml(): Seq[Node] = Seq.empty + + /** + * Called when an application UI is unloaded from the history server. + */ + def onUIDetached(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index f4235df245128..48f3bd37c32f6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,14 +17,18 @@ package org.apache.spark.deploy.history -import java.io.{FileNotFoundException, IOException, OutputStream} -import java.util.UUID +import java.io.{File, FileNotFoundException, IOException} +import java.nio.file.Files +import java.nio.file.attribute.PosixFilePermissions +import java.util.{Date, ServiceLoader, UUID} import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.xml.Node +import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, Path} @@ -35,9 +39,15 @@ import org.apache.hadoop.security.AccessControlException import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging +import org.apache.spark.kvstore._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.KVUtils +import org.apache.spark.status.KVUtils._ +import org.apache.spark.status.api.v1 import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} @@ -78,6 +88,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) this(conf, new SystemClock()) } + import config._ import FsHistoryProvider._ // Interval between safemode checks. @@ -94,8 +105,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val NUM_PROCESSING_THREADS = conf.getInt(SPARK_HISTORY_FS_NUM_REPLAY_THREADS, Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) - private val logDir = conf.getOption("spark.history.fs.logDirectory") - .getOrElse(DEFAULT_LOG_DIR) + private val logDir = conf.get(EVENT_LOG_DIR) private val HISTORY_UI_ACLS_ENABLE = conf.getBoolean("spark.history.ui.acls.enable", false) private val HISTORY_UI_ADMIN_ACLS = conf.get("spark.history.ui.admin.acls", "") @@ -117,17 +127,28 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // used for logging msgs (logs are re-scanned based on file size, rather than modtime) private val lastScanTime = new java.util.concurrent.atomic.AtomicLong(-1) - // Mapping of application IDs to their metadata, in descending end time order. Apps are inserted - // into the map in order, so the LinkedHashMap maintains the correct ordering. - @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] - = new mutable.LinkedHashMap() + private val pendingReplayTasksCount = new java.util.concurrent.atomic.AtomicInteger(0) - val fileToAppInfo = new mutable.HashMap[Path, FsApplicationAttemptInfo]() + private val storePath = new File(conf.get(LOCAL_STORE_DIR)) + require(storePath.isDirectory(), s"Configured store directory ($storePath) does not exist.") - // List of application logs to be deleted by event log cleaner. - private var attemptsToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] + private val listing = { + val metadata = new KVStoreMetadata(CURRENT_VERSION, AppStateStore.CURRENT_VERSION, + logDir.toString()) + try { + open(new File(storePath, "listing.ldb"), metadata) + } catch { + case e: Exception => + // If there's an error, remove the listing database and any existing UI database + // from the store directory, since it's extremely likely that they'll all contain + // incompatible information. + logWarning(s"Error while opening existing listing database, creating new one.", e) + storePath.listFiles().foreach(Utils.deleteRecursively) + open(new File(storePath, "listing.ldb"), metadata) + } + } - private val pendingReplayTasksCount = new java.util.concurrent.atomic.AtomicInteger(0) + private val activeUIs = new mutable.HashMap[(String, Option[String]), LoadedAppUI]() /** * Return a runnable that performs the given operation on the event logs. @@ -152,7 +173,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - // Conf option used for testing the initialization code. val initThread = initialize() private[history] def initialize(): Thread = { @@ -229,10 +249,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - override def getListing(): Iterator[FsApplicationHistoryInfo] = applications.values.iterator + override def getListing(): Iterator[ApplicationHistoryInfo] = { + listing.view(classOf[KVStoreApplicationInfo]) + .index("endTime") + .reverse() + .iterator() + .asScala + .map(_.toAppHistoryInfo) + } - override def getApplicationInfo(appId: String): Option[FsApplicationHistoryInfo] = { - applications.get(appId) + override def getApplicationInfo(appId: String): Option[ApplicationHistoryInfo] = { + try { + Some(load(appId).toAppHistoryInfo()) + } catch { + case e: NoSuchElementException => + None + } } override def getEventLogsUnderProcess(): Int = pendingReplayTasksCount.get() @@ -240,42 +272,69 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { - try { - applications.get(appId).flatMap { appInfo => - appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => - val replayBus = new ReplayListenerBus() - val ui = { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - SparkUI.createHistoryUI(conf, replayBus, appSecManager, appInfo.name, - HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) - // Do not call ui.bind() to avoid creating a new server for each application - } + val app = try { + load(appId) + } catch { + case _: NoSuchElementException => + return None + } - val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) - - val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) - - if (appListener.appId.isDefined) { - ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) - // make sure to set admin acls before view acls so they are properly picked up - val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") - ui.getSecurityManager.setAdminAcls(adminAcls) - ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) - val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - appListener.adminAclsGroups.getOrElse("") - ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) - ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) - Some(LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize))) - } else { - None - } + val attempt = app.attempts.find(_.info.attemptId == attemptId).orNull + if (attempt == null) { + return None + } - } + val conf = this.conf.clone() + val secManager = new SecurityManager(conf) + + secManager.setAcls(HISTORY_UI_ACLS_ENABLE) + // make sure to set admin acls before view acls so they are properly picked up + secManager.setAdminAcls(HISTORY_UI_ADMIN_ACLS + "," + attempt.adminAcls.getOrElse("")) + secManager.setViewAcls(attempt.info.sparkUser, attempt.viewAcls.getOrElse("")) + secManager.setAdminAclsGroups(HISTORY_UI_ADMIN_ACLS_GROUPS + "," + + attempt.adminAclsGroups.getOrElse("")) + secManager.setViewAclsGroups(attempt.viewAclsGroups.getOrElse("")) + + val path = uiStorePath(appId, attemptId) + if (!path.exists()) { + throw new IllegalStateException( + s"Application entry for $appId / $attemptId found, but UI not available.") + } + + val replayBus = new ReplayListenerBus() + + // Create the UI under a lock so that a valid disk store is used, in case the update thread + // is writing a new disk store for the application (see replaceStore()). + val loadedUI = synchronized { + val ui = SparkUI.create(None, AppStateStore.loadStore(conf, path), conf, secManager, + app.info.name, + HistoryServer.getAttemptURI(appId, attempt.info.attemptId), + attempt.info.startTime.getTime()) + val loaded = LoadedAppUI(ui) + activeUIs((appId, attemptId)) = loaded + loaded + } + + // TODO: remove the following replay code. It's currently needed because not all of the + // UI uses the cached state store. Once that's done (i.e. after the SQL UI is ported + // over), this code can be removed. + try { + val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory], + Utils.getContextOrSparkClassLoader).asScala + listenerFactories.foreach { listenerFactory => + val listeners = listenerFactory.createListeners(conf, loadedUI.ui) + listeners.foreach(replayBus.addListener) } + + val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) + replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) } catch { - case e: FileNotFoundException => None + case e: Exception => + onUIDetached(appId, attemptId, loadedUI.ui) + throw e } + + Some(loadedUI) } override def getEmptyListingHtml(): Seq[Node] = { @@ -298,11 +357,34 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) Map("Event log directory" -> logDir.toString) ++ safeMode } - override def stop(): Unit = { + override def stop(): Unit = synchronized { if (initThread != null && initThread.isAlive()) { initThread.interrupt() initThread.join() } + Seq(pool, replayExecutor).foreach { executor => + executor.shutdown() + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + executor.shutdownNow() + } + } + activeUIs.foreach { case (_, loadedUI) => loadedUI.ui.store.close() } + activeUIs.clear() + listing.close() + } + + override def onUIDetached(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { + val uiOption = synchronized { + activeUIs.remove((appId, attemptId)) + } + uiOption.foreach { loadedUI => + loadedUI.lock.writeLock().lock() + try { + loadedUI.ui.store.close() + } finally { + loadedUI.lock.writeLock().unlock() + } + } } /** @@ -314,24 +396,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { val newLastScanTime = getNewLastScanTime() logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") - val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) - .getOrElse(Seq[FileStatus]()) + val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) // scan for modified applications, replay and merge them - val logInfos: Seq[FileStatus] = statusList + val logInfos = statusList .filter { entry => - val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L) !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would log an error which can be scary to // the end-user. !entry.getPath().getName().startsWith(".") && - prevFileSize < entry.getLen() && - SparkHadoopUtil.get.checkAccessPermission(entry, FsAction.READ) + SparkHadoopUtil.get.checkAccessPermission(entry, FsAction.READ) && + recordedFileSize(entry.getPath()) < entry.getLen() } .flatMap { entry => Some(entry) } .sortWith { case (entry1, entry2) => - entry1.getModificationTime() >= entry2.getModificationTime() - } + entry1.getModificationTime() > entry2.getModificationTime() + } if (logInfos.nonEmpty) { logDebug(s"New/updated attempts found: ${logInfos.size} ${logInfos.map(_.getPath)}") @@ -419,216 +499,126 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - applications.get(appId) match { - case Some(appInfo) => - try { - // If no attempt is specified, or there is no attemptId for attempts, return all attempts - appInfo.attempts.filter { attempt => - attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get - }.foreach { attempt => - val logPath = new Path(logDir, attempt.logPath) - zipFileToStream(logPath, attempt.logPath, zipStream) - } - } finally { - zipStream.close() + val app = try { + load(appId) + } catch { + case _: NoSuchElementException => + throw new SparkException(s"Logs for $appId not found.") + } + + try { + // If no attempt is specified, or there is no attemptId for attempts, return all attempts + attemptId + .map { id => app.attempts.filter(_.info.attemptId == Some(id)) } + .getOrElse(app.attempts) + .map(_.logPath) + .foreach { log => + zipFileToStream(new Path(logDir, log), log, zipStream) } - case None => throw new SparkException(s"Logs for $appId not found.") + } finally { + zipStream.close() } } /** - * Replay the log files in the list and merge the list of old applications with new ones + * Replay the given log file, saving the application in the listing db. */ protected def mergeApplicationListing(fileStatus: FileStatus): Unit = { - val newAttempts = try { - val eventsFilter: ReplayEventsFilter = { eventString => - eventString.startsWith(APPL_START_EVENT_PREFIX) || - eventString.startsWith(APPL_END_EVENT_PREFIX) - } + val logPath = fileStatus.getPath() + logInfo(s"Replaying log path: $logPath") - val logPath = fileStatus.getPath() - val appCompleted = isApplicationCompleted(fileStatus) - - // Use loading time as lastUpdated since some filesystems don't update modifiedTime - // each time file is updated. However use modifiedTime for completed jobs so lastUpdated - // won't change whenever HistoryServer restarts and reloads the file. - val lastUpdated = if (appCompleted) fileStatus.getModificationTime else clock.getTimeMillis() - - val appListener = replay(fileStatus, appCompleted, new ReplayListenerBus(), eventsFilter) - - // Without an app ID, new logs will render incorrectly in the listing page, so do not list or - // try to show their UI. - if (appListener.appId.isDefined) { - val attemptInfo = new FsApplicationAttemptInfo( - logPath.getName(), - appListener.appName.getOrElse(NOT_STARTED), - appListener.appId.getOrElse(logPath.getName()), - appListener.appAttemptId, - appListener.startTime.getOrElse(-1L), - appListener.endTime.getOrElse(-1L), - lastUpdated, - appListener.sparkUser.getOrElse(NOT_STARTED), - appCompleted, - fileStatus.getLen() - ) - fileToAppInfo(logPath) = attemptInfo - logDebug(s"Application log ${attemptInfo.logPath} loaded successfully: $attemptInfo") - Some(attemptInfo) - } else { - logWarning(s"Failed to load application log ${fileStatus.getPath}. " + - "The application may have not started.") - None - } + val bus = new ReplayListenerBus() + val listener = new AppListingListener(fileStatus, clock) + bus.addListener(listener) + + // Write the UI data to a temp location. + val tempUiPath = createTempDir("uistore") + val store = AppStateStore.createStore(tempUiPath, conf, bus) + val appCompleted = isApplicationCompleted(fileStatus) + val logInput = EventLoggingListener.openEventLog(logPath, fs) + try { + bus.replay(logInput, logPath.toString, !appCompleted) + store.close() } catch { case e: Exception => - logError( - s"Exception encountered when attempting to load application log ${fileStatus.getPath}", - e) - None + store.close() + Utils.deleteRecursively(tempUiPath) + throw e + } finally { + logInput.close() } - if (newAttempts.isEmpty) { - return - } - - // Build a map containing all apps that contain new attempts. The app information in this map - // contains both the new app attempt, and those that were already loaded in the existing apps - // map. If an attempt has been updated, it replaces the old attempt in the list. - val newAppMap = new mutable.HashMap[String, FsApplicationHistoryInfo]() - - applications.synchronized { - newAttempts.foreach { attempt => - val appInfo = newAppMap.get(attempt.appId) - .orElse(applications.get(attempt.appId)) - .map { app => - val attempts = - app.attempts.filter(_.attemptId != attempt.attemptId) ++ List(attempt) - new FsApplicationHistoryInfo(attempt.appId, attempt.name, - attempts.sortWith(compareAttemptInfo)) - } - .getOrElse(new FsApplicationHistoryInfo(attempt.appId, attempt.name, List(attempt))) - newAppMap(attempt.appId) = appInfo - } - - // Merge the new app list with the existing one, maintaining the expected ordering (descending - // end time). Maintaining the order is important to avoid having to sort the list every time - // there is a request for the log list. - val newApps = newAppMap.values.toSeq.sortWith(compareAppInfo) - val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { - if (!mergedApps.contains(info.id)) { - mergedApps += (info.id -> info) - } - } - - val newIterator = newApps.iterator.buffered - val oldIterator = applications.values.iterator.buffered - while (newIterator.hasNext && oldIterator.hasNext) { - if (newAppMap.contains(oldIterator.head.id)) { - oldIterator.next() - } else if (compareAppInfo(newIterator.head, oldIterator.head)) { - addIfAbsent(newIterator.next()) - } else { - addIfAbsent(oldIterator.next()) - } - } - newIterator.foreach(addIfAbsent) - oldIterator.foreach(addIfAbsent) - - applications = mergedApps + // Move the UI store to its final location if the app ID is known, otherwise discard it. + listener.applicationInfo.foreach { app => + addListing(app) + replaceStore(app.info.id, app.attempts.head.info.attemptId, tempUiPath) } + Utils.deleteRecursively(tempUiPath) + listing.write(new LogInfo(logPath.toString(), fileStatus.getLen())) + logInfo(s"Finished parsing $logPath") } /** * Delete event logs from the log directory according to the clean policy defined by the user. */ private[history] def cleanLogs(): Unit = { + var iterator: Option[KVStoreIterator[KVStoreApplicationInfo]] = None try { - val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 - - val now = clock.getTimeMillis() - val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - - def shouldClean(attempt: FsApplicationAttemptInfo): Boolean = { - now - attempt.lastUpdated > maxAge - } - - // Scan all logs from the log directory. - // Only completed applications older than the specified max age will be deleted. - applications.values.foreach { app => - val (toClean, toRetain) = app.attempts.partition(shouldClean) - attemptsToClean ++= toClean - - if (toClean.isEmpty) { - appsToRetain += (app.id -> app) - } else if (toRetain.nonEmpty) { - appsToRetain += (app.id -> - new FsApplicationHistoryInfo(app.id, app.name, toRetain.toList)) + val maxTime = clock.getTimeMillis() - conf.get(MAX_LOG_AGE_S) * 1000 + + // Iterate descending over all applications whose oldest attempt is older than the maxAge. + iterator = Some(listing.view(classOf[KVStoreApplicationInfo]) + .index("oldestAttempt") + .reverse() + .first(maxTime) + .closeableIterator()) + + iterator.get.asScala.foreach { app => + val (remaining, toDelete) = app.attempts.partition { attempt => + attempt.info.lastUpdated.getTime() >= maxTime + } + if (remaining.nonEmpty) { + val newApp = new KVStoreApplicationInfo(app.info, remaining) + listing.write(newApp) + } else { + listing.delete(app.getClass(), app.id) } - } - - applications = appsToRetain - val leftToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] - attemptsToClean.foreach { attempt => - try { - fs.delete(new Path(logDir, attempt.logPath), true) - } catch { - case e: AccessControlException => - logInfo(s"No permission to delete ${attempt.logPath}, ignoring.") - case t: IOException => - logError(s"IOException in cleaning ${attempt.logPath}", t) - leftToClean += attempt + toDelete.foreach { attempt => + val logPath = new Path(logDir, attempt.logPath) + try { + listing.delete(classOf[LogInfo], logPath.toString()) + } catch { + case _: NoSuchElementException => + logDebug(s"Log info entry for $logPath not found.") + } + try { + fs.delete(logPath, true) + } catch { + case e: AccessControlException => + logInfo(s"No permission to delete ${attempt.logPath}, ignoring.") + case t: IOException => + logError(s"IOException in cleaning ${attempt.logPath}", t) + } } } - - attemptsToClean = leftToClean } catch { - case t: Exception => logError("Exception in cleaning logs", t) + case t: Exception => logError("Exception while cleaning logs", t) + } finally { + iterator.foreach(_.close()) } } /** - * Comparison function that defines the sort order for the application listing. - * - * @return Whether `i1` should precede `i2`. - */ - private def compareAppInfo( - i1: FsApplicationHistoryInfo, - i2: FsApplicationHistoryInfo): Boolean = { - val a1 = i1.attempts.head - val a2 = i2.attempts.head - if (a1.endTime != a2.endTime) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime - } - - /** - * Comparison function that defines the sort order for application attempts within the same - * application. Order is: attempts are sorted by descending start time. - * Most recent attempt state matches with current state of the app. - * - * Normally applications should have a single running attempt; but failure to call sc.stop() - * may cause multiple running attempts to show up. - * - * @return Whether `a1` should precede `a2`. - */ - private def compareAttemptInfo( - a1: FsApplicationAttemptInfo, - a2: FsApplicationAttemptInfo): Boolean = { - a1.startTime >= a2.startTime - } - - /** - * Replays the events in the specified log file on the supplied `ReplayListenerBus`. Returns - * an `ApplicationEventListener` instance with event data captured from the replay. - * `ReplayEventsFilter` determines what events are replayed and can therefore limit the - * data captured in the returned `ApplicationEventListener` instance. + * Replays the events in the specified log file on the supplied `ReplayListenerBus`. + * `ReplayEventsFilter` determines what events are replayed. */ private def replay( eventLog: FileStatus, appCompleted: Boolean, bus: ReplayListenerBus, - eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): ApplicationEventListener = { + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { val logPath = eventLog.getPath() logInfo(s"Replaying log path: $logPath") // Note that the eventLog may have *increased* in size since when we grabbed the filestatus, @@ -639,10 +629,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // after it's created, so we get a file size that is no bigger than what is actually read. val logInput = EventLoggingListener.openEventLog(logPath, fs) try { - val appListener = new ApplicationEventListener - bus.addListener(appListener) bus.replay(logInput, logPath.toString, !appCompleted, eventsFilter) - appListener } finally { logInput.close() } @@ -678,56 +665,94 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * @return a summary of the component state */ override def toString: String = { - val header = s""" - | FsHistoryProvider: logdir=$logDir, - | last scan time=$lastScanTime - | Cached application count =${applications.size}} - """.stripMargin - val sb = new StringBuilder(header) - applications.foreach(entry => sb.append(entry._2).append("\n")) - sb.toString + val count = listing.count(classOf[KVStoreApplicationInfo]) + s"""|FsHistoryProvider{logdir=$logDir, + | storedir=$storePath, + | last scan time=$lastScanTime + | application count=$count}""".stripMargin } - /** - * Look up an application attempt - * @param appId application ID - * @param attemptId Attempt ID, if set - * @return the matching attempt, if found - */ - def lookup(appId: String, attemptId: Option[String]): Option[FsApplicationAttemptInfo] = { - applications.get(appId).flatMap { appInfo => - appInfo.attempts.find(_.attemptId == attemptId) + private def recordedFileSize(log: Path): Long = { + try { + listing.read(classOf[LogInfo], log.toString()).fileSize + } catch { + case _: NoSuchElementException => 0L } } + private def load(appId: String): KVStoreApplicationInfo = { + listing.read(classOf[KVStoreApplicationInfo], appId) + } + /** - * Return true iff a newer version of the UI is available. The check is based on whether the - * fileSize for the currently loaded UI is smaller than the file size the last time - * the logs were loaded. - * - * This is a very cheap operation -- the work of loading the new attempt was already done - * by [[checkForLogs]]. - * @param appId application to probe - * @param attemptId attempt to probe - * @param prevFileSize the file size of the logs for the currently displayed UI + * Write the app's information to the given store. Serialized to avoid the (notedly rare) case + * where two threads are processing separate attempts of the same application. */ - private def updateProbe( - appId: String, - attemptId: Option[String], - prevFileSize: Long)(): Boolean = { - lookup(appId, attemptId) match { - case None => - logDebug(s"Application Attempt $appId/$attemptId not found") - false - case Some(latest) => - prevFileSize < latest.fileSize + private def addListing(app: KVStoreApplicationInfo): Unit = listing.synchronized { + val attempt = app.attempts.head + + val oldApp = try { + listing.read(classOf[KVStoreApplicationInfo], app.id) + } catch { + case _: NoSuchElementException => + app + } + + def compareAttemptInfo(a1: KVStoreAttemptInfo, a2: KVStoreAttemptInfo): Boolean = { + a1.info.startTime.getTime() > a2.info.startTime.getTime() + } + + val attempts = oldApp.attempts.filter(_.info.attemptId != attempt.info.attemptId) ++ + List(attempt) + val oldestAttempt = attempts.map(_.info.lastUpdated.getTime()).min + + val newAppInfo = new KVStoreApplicationInfo( + app.info, + attempts.sortWith(compareAttemptInfo)) + listing.write(newAppInfo) + } + + private def createTempDir(name: String): File = { + val perms = PosixFilePermissions.fromString("rwx------") + Files.createTempDirectory(storePath.toPath(), name, + PosixFilePermissions.asFileAttribute(perms)).toFile() + } + + private def uiStorePath(appId: String, attemptId: Option[String]): File = { + val fileName = appId + attemptId.map("_" + _).getOrElse("") + ".ldb" + new File(storePath, fileName) + } + + private def replaceStore(appId: String, attemptId: Option[String], newStore: File): Unit = { + val uiStore = uiStorePath(appId, attemptId) + + synchronized { + // If there's an active UI for the application, invalidate it and close its store, so that + // we can replace it with the updated one. + activeUIs.remove((appId, attemptId)).foreach { loadedUI => + loadedUI.invalidate() + loadedUI.ui.store.close() + } + + if (uiStore.exists()) { + Utils.deleteRecursively(uiStore) + } + + if (!newStore.renameTo(uiStore)) { + throw new IOException(s"Failed to rename UI store from $newStore to $uiStore.") + } } } + + /** For testing. Returns internal data about a single attempt. */ + private[history] def getAttempt(appId: String, attemptId: Option[String]): KVStoreAttemptInfo = { + load(appId).attempts.find(_.info.attemptId == attemptId).getOrElse( + throw new NoSuchElementException(s"Cannot find attempt $attemptId of $appId.")) + } + } private[history] object FsHistoryProvider { - val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - private val NOT_STARTED = "" private val SPARK_HISTORY_FS_NUM_REPLAY_THREADS = "spark.history.fs.numReplayThreads" @@ -735,52 +760,153 @@ private[history] object FsHistoryProvider { private val APPL_START_EVENT_PREFIX = "{\"Event\":\"SparkListenerApplicationStart\"" private val APPL_END_EVENT_PREFIX = "{\"Event\":\"SparkListenerApplicationEnd\"" + + private val CURRENT_VERSION = 1L } -/** - * Application attempt information. - * - * @param logPath path to the log file, or, for a legacy log, its directory - * @param name application name - * @param appId application ID - * @param attemptId optional attempt ID - * @param startTime start time (from playback) - * @param endTime end time (from playback). -1 if the application is incomplete. - * @param lastUpdated the modification time of the log file when this entry was built by replaying - * the history. - * @param sparkUser user running the application - * @param completed flag to indicate whether or not the application has completed. - * @param fileSize the size of the log file the last time the file was scanned for changes - */ -private class FsApplicationAttemptInfo( +private[history] case class KVStoreMetadata( + val version: Long, + val uiVersion: Long, + val logDir: String) + +private[history] case class LogInfo( + @KVIndexParam val logPath: String, + val fileSize: Long) + +private[history] class KVStoreAttemptInfo( + val info: v1.ApplicationAttemptInfo, val logPath: String, - val name: String, - val appId: String, - attemptId: Option[String], - startTime: Long, - endTime: Long, - lastUpdated: Long, - sparkUser: String, - completed: Boolean, - val fileSize: Long) - extends ApplicationAttemptInfo( - attemptId, startTime, endTime, lastUpdated, sparkUser, completed) { - - /** extend the superclass string value with the extra attributes of this class */ - override def toString: String = { - s"FsApplicationAttemptInfo($name, $appId," + - s" ${super.toString}, source=$logPath, size=$fileSize" + val fileSize: Long, + val adminAcls: Option[String], + val viewAcls: Option[String], + val adminAclsGroups: Option[String], + val viewAclsGroups: Option[String]) { + + def toAppAttemptInfo(): ApplicationAttemptInfo = { + ApplicationAttemptInfo(info.attemptId, info.startTime.getTime(), + info.endTime.getTime(), info.lastUpdated.getTime(), info.sparkUser, + info.completed) } + } -/** - * Application history information - * @param id application ID - * @param name application name - * @param attempts list of attempts, most recent first. - */ -private class FsApplicationHistoryInfo( - id: String, - override val name: String, - override val attempts: List[FsApplicationAttemptInfo]) - extends ApplicationHistoryInfo(id, name, attempts) +private[history] class KVStoreApplicationInfo( + val info: v1.ApplicationInfo, + val attempts: List[KVStoreAttemptInfo]) { + + @JsonIgnore + @KVIndexParam + def id: String = info.id + + @JsonIgnore + @KVIndexParam("endTime") + def endTime(): Long = attempts.head.info.endTime.getTime() + + @JsonIgnore + @KVIndexParam("oldestAttempt") + def oldestAttempt(): Long = attempts.map(_.info.lastUpdated.getTime()).min + + @JsonIgnore + def toAppHistoryInfo(): ApplicationHistoryInfo = { + ApplicationHistoryInfo(info.id, info.name, attempts.map(_.toAppAttemptInfo())) + } + + @JsonIgnore + def toApiInfo(): v1.ApplicationInfo = { + new v1.ApplicationInfo(info.id, info.name, info.coresGranted, info.maxCores, + info.coresPerExecutor, info.memoryPerExecutorMB, attempts.map(_.info)) + } + +} + +private[history] class AppListingListener( + log: FileStatus, + clock: Clock) + extends SparkListener with KVUtils { + + // This listener doesn't use read/update, so no need for a KVStore instance. + override protected val kvstore: KVStore = null + + private var attempt = newAttemptInfo(None, + new v1.ApplicationAttemptInfo(None, new Date(-1), new Date(-1), + new Date(log.getModificationTime()), -1, null, false), + log.getPath().getName(), log.getLen()) + + private var app = new KVStoreApplicationInfo( + new v1.ApplicationInfo(null, null, None, None, None, None, Nil), + Nil) + + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { + val appId = event.appId.orNull + app = new KVStoreApplicationInfo( + new v1.ApplicationInfo(appId, event.appName, app.info.coresGranted, app.info.maxCores, + app.info.coresPerExecutor, app.info.memoryPerExecutorMB, Nil), + Nil) + + val newInfo = new v1.ApplicationAttemptInfo( + event.appAttemptId, + new Date(event.time), + attempt.info.endTime, + new Date(clock.getTimeMillis()), + attempt.info.duration, + event.sparkUser, + attempt.info.completed) + + attempt = newAttemptInfo(attempt, info = newInfo) + } + + override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { + val newInfo = new v1.ApplicationAttemptInfo( + attempt.info.attemptId, + attempt.info.startTime, + new Date(event.time), + new Date(log.getModificationTime()), + event.time - attempt.info.startTime.getTime(), + attempt.info.sparkUser, + true) + + attempt = newAttemptInfo(attempt, info = newInfo) + } + + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { + val allProperties = event.environmentDetails("Spark Properties").toMap + val viewAcls = allProperties.get("spark.ui.view.acls") + val adminAcls = allProperties.get("spark.admin.acls") + val viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") + val adminAclsGroups = allProperties.get("spark.admin.acls.groups") + + attempt = newAttemptInfo(attempt, + viewAcls = viewAcls, + adminAcls = adminAcls, + viewAclsGroups = viewAclsGroups, + adminAclsGroups = adminAclsGroups) + } + + def applicationInfo: Option[KVStoreApplicationInfo] = { + if (app.id != null) { + Some(new KVStoreApplicationInfo(app.info, List(attempt))) + } else { + None + } + } + + private def newAttemptInfo( + old: Option[KVStoreAttemptInfo], + info: Option[v1.ApplicationAttemptInfo] = None, + logPath: Option[String] = None, + fileSize: Option[Long] = None, + adminAcls: Option[Option[String]] = None, + viewAcls: Option[Option[String]] = None, + adminAclsGroups: Option[Option[String]] = None, + viewAclsGroups: Option[Option[String]] = None): KVStoreAttemptInfo = { + new KVStoreAttemptInfo( + info.orElse(old.map(_.info)).orNull, + logPath.orElse(old.map(_.logPath)).orNull, + fileSize.orElse(old.map(_.fileSize)).getOrElse(0L), + adminAcls.getOrElse(old.flatMap(_.adminAcls)), + viewAcls.getOrElse(old.flatMap(_.viewAcls)), + adminAclsGroups.getOrElse(old.flatMap(_.adminAclsGroups)), + viewAclsGroups.getOrElse(old.flatMap(_.viewAclsGroups))) + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d9c8fda99ef97..4900a8130a921 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -106,8 +106,8 @@ class HistoryServer( } } - def getSparkUI(appKey: String): Option[SparkUI] = { - appCache.getSparkUI(appKey) + override def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T = { + appCache.withSparkUI(appId, attemptId)(fn) } initialize() @@ -140,7 +140,6 @@ class HistoryServer( override def stop() { super.stop() provider.stop() - appCache.stop() } /** Attach a reconstructed UI to this server. Only valid after bind(). */ @@ -158,6 +157,7 @@ class HistoryServer( override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) + provider.onUIDetached(appId, attemptId, ui) } /** @@ -224,15 +224,13 @@ class HistoryServer( */ private def loadAppUi(appId: String, attemptId: Option[String]): Boolean = { try { - appCache.get(appId, attemptId) + appCache.withSparkUI(appId, attemptId) { _ => + // Do nothing, just force the UI to load. + } true } catch { - case NonFatal(e) => e.getCause() match { - case nsee: NoSuchElementException => - false - - case cause: Exception => throw cause - } + case NonFatal(e: NoSuchElementException) => + false } } @@ -325,7 +323,7 @@ object HistoryServer extends Logging { } } - private[history] def getAttemptURI(appId: String, attemptId: Option[String]): String = { + private[spark] def getAttemptURI(appId: String, attemptId: Option[String]): String = { val attemptSuffix = attemptId.map { id => s"/$id" }.getOrElse("") s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/deploy/history/config.scala similarity index 57% rename from core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala rename to core/src/main/scala/org/apache/spark/deploy/history/config.scala index 237aeac185877..9ca07e3d63271 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/config.scala @@ -14,21 +14,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.status.api.v1 -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType +package org.apache.spark.deploy.history -import org.apache.spark.ui.SparkUI +import java.util.concurrent.TimeUnit -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneRDDResource(ui: SparkUI) { +import org.apache.spark.internal.config.ConfigBuilder - @GET - def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = { - AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse( - throw new NotFoundException(s"no rdd found w/ id $rddId") - ) - } +private[spark] object config { + + val DEFAULT_LOG_DIR = "file:/tmp/spark-events" + + val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory") + .stringConf + .createWithDefault(DEFAULT_LOG_DIR) + + val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("7d") + + val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path") + .stringConf + .createWithDefault("/var/lib/spark-history") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala deleted file mode 100644 index 28c45d800ed06..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ /dev/null @@ -1,60 +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 - -/** - * A simple listener for application events. - * - * This listener expects to hear events from a single application only. If events - * from multiple applications are seen, the behavior is unspecified. - */ -private[spark] class ApplicationEventListener extends SparkListener { - var appName: Option[String] = None - var appId: Option[String] = None - var appAttemptId: Option[String] = None - var sparkUser: Option[String] = None - var startTime: Option[Long] = None - var endTime: Option[Long] = None - var viewAcls: Option[String] = None - var adminAcls: Option[String] = None - var viewAclsGroups: Option[String] = None - var adminAclsGroups: Option[String] = None - - override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - appName = Some(applicationStart.appName) - appId = applicationStart.appId - appAttemptId = applicationStart.appAttemptId - startTime = Some(applicationStart.time) - sparkUser = Some(applicationStart.sparkUser) - } - - override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - endTime = Some(applicationEnd.time) - } - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - val environmentDetails = environmentUpdate.environmentDetails - val allProperties = environmentDetails("Spark Properties").toMap - viewAcls = allProperties.get("spark.ui.view.acls") - adminAcls = allProperties.get("spark.admin.acls") - viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") - adminAclsGroups = allProperties.get("spark.admin.acls.groups") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index b25a4bfb501fb..70fafa9b0b303 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,6 +17,10 @@ package org.apache.spark.scheduler.cluster +import scala.annotation.meta.getter + +import com.fasterxml.jackson.annotation.JsonIgnore + import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} /** @@ -29,9 +33,12 @@ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} * @param totalCores The total number of cores available to the executor */ private[cluster] class ExecutorData( + @(JsonIgnore @getter) val executorEndpoint: RpcEndpointRef, + @(JsonIgnore @getter) val executorAddress: RpcAddress, override val executorHost: String, + @(JsonIgnore @getter) var freeCores: Int, override val totalCores: Int, override val logUrlMap: Map[String, String] diff --git a/core/src/main/scala/org/apache/spark/status/AppStateListener.scala b/core/src/main/scala/org/apache/spark/status/AppStateListener.scala new file mode 100644 index 0000000000000..40fc309d3ffae --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppStateListener.scala @@ -0,0 +1,1466 @@ +/* + * 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.status + +import java.util.Date +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap, HashSet} + +import com.codahale.metrics.{MetricRegistry, Slf4jReporter} +import org.slf4j.LoggerFactory + +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.kvstore.KVStore +import org.apache.spark.scheduler._ +import org.apache.spark.status.api.v1 +import org.apache.spark.storage._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.scope._ +import org.apache.spark.util.{AccumulatorContext, ThreadUtils} + +/** + * A Spark listener that writes application information to a data store. The types written to the + * store are defined in the `storeTypes.scala` file and are based on the public REST API. + * + * TODO (future): + * - Add support for cleaning up old data so that the store doesn't grow arbitrarily. This is + * especially important for providing an in-memory store for when the JNI libraries are not + * available. + * + * TODO (future): + * - to enable incremental parsing of event logs, all state in this class needs to be serialized + * to the underlying store and loaded when the class is instantiated. That could potentially + * be written to the AppStatusStoreMetadata object. + */ +private class AppStateListener( + override protected val kvstore: KVStore, + conf: SparkConf) extends SparkListener with KVUtils { + + import config._ + import AppStateListener._ + + private var appId: String = null + private var activeJobs: Set[Int] = Set() + private var executorEventId: Long = 0L + + // Keep a list of active executor to hosts. The blacklist event handlers need to iterate + // over existing executors when blacklisting nodes, and because AsyncCachingStore may not + // reflect writes in iterators immediately, some executors might be missed otherwise. + private var activeExecutorIds = new HashMap[String, HashSet[String]]() + + private var coresPerTask: Int = 1 + + private val metrics = new MetricRegistry() + private val anyEventTimer = metrics.timer("anyEvent") + + private val metricsExecutor = if (conf.get(LISTENER_METRICS_LOG)) { + val executor = ThreadUtils.newDaemonSingleThreadScheduledExecutor("app-state-store-metrics") + val task = new Runnable() { + override def run(): Unit = dumpMetrics(Slf4jReporter.LoggingLevel.INFO) + } + val period = conf.get(METRICS_LOG_PERIOD) + executor.scheduleAtFixedRate(task, period, period, TimeUnit.SECONDS) + Some(executor) + } else { + None + } + + def stop(): Unit = { + metricsExecutor.foreach(_.shutdown()) + + // Dump the metrics one last time. + val level = if (metricsExecutor.isDefined) { + Slf4jReporter.LoggingLevel.INFO + } else { + Slf4jReporter.LoggingLevel.DEBUG + } + dumpMetrics(level) + } + + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { + assert(event.appId.isDefined, "Application without IDs are not supported.") + this.appId = event.appId.get + + val attempt = new v1.ApplicationAttemptInfo( + event.appAttemptId, + new Date(event.time), + DEFAULT_DATE, + new Date(event.time), + -1L, + event.sparkUser, + false) + + val app = new v1.ApplicationInfo( + appId, + event.appName, + None, + None, + None, + None, + Seq(attempt)) + + val stored = new ApplicationInfoWrapper(app) + kvstore.write(stored) + } + + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { + val details = event.environmentDetails + + val jvmInfo = Map(details("JVM Information"): _*) + val runtime = new v1.RuntimeInfo( + jvmInfo("Java Version"), + jvmInfo("Java Home"), + jvmInfo("Scala Version")) + + val envInfo = new v1.ApplicationEnvironmentInfo( + runtime, + details("Spark Properties"), + details("System Properties"), + details("Classpath Entries")) + + coresPerTask = envInfo.sparkProperties.toMap.get("spark.task.cpus").map(_.toInt) + .getOrElse(coresPerTask) + kvstore.write(new ApplicationEnvironmentInfoWrapper(envInfo)) + } + + override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { + update[ApplicationInfoWrapper](appId) { wrapper => + val app = wrapper.get.info + val attempt = app.attempts.head + + val newAttempt = new v1.ApplicationAttemptInfo( + attempt.attemptId, + attempt.startTime, + new Date(event.time), + new Date(event.time), + event.time - attempt.startTime.getTime(), + attempt.sparkUser, + true) + + val newApp = new v1.ApplicationInfo( + app.id, app.name, app.coresGranted, app.maxCores, app.coresPerExecutor, + app.memoryPerExecutorMB, Seq(newAttempt)) + + new ApplicationInfoWrapper(newApp) + } + } + + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = time(event) { + // This needs to be an update in case an executor re-registers after the driver has + // marked it as "dead". + updateExecutorSummary(event.executorId) { exec => + val newInfo = newExecutorSummary(exec.info, + hostPort = event.executorInfo.executorHost, + isActive = true, + totalCores = event.executorInfo.totalCores, + maxTasks = event.executorInfo.totalCores / coresPerTask, + executorLogs = event.executorInfo.logUrlMap) + + val newWrapper = new ExecutorSummaryWrapper(newInfo) + activeExecutorIds.getOrElseUpdate(newWrapper.host, new HashSet()) += newWrapper.id + newWrapper + } + + writeExecutorEvent(event) + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = time(event) { + updateExecutorSummary(event.executorId) { exec => + val newInfo = newExecutorSummary(exec.info, + isActive = false) + val newWrapper = new ExecutorSummaryWrapper(newInfo) + activeExecutorIds.getOrElseUpdate(newWrapper.host, new HashSet()) -= newWrapper.id + newWrapper + } + + writeExecutorEvent(event) + } + + override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = time(event) { + updateBlackListStatus(event.executorId, true) + } + + private def writeExecutorEvent(event: SparkListenerEvent): Unit = { + executorEventId += 1 + kvstore.write(new ExecutorEventData(executorEventId, event)) + } + + override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { + time(event) { + updateBlackListStatus(event.executorId, false) + } + } + + override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = time(event) { + updateNodeBlackList(event.hostId, true) + } + + override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = time(event) { + updateNodeBlackList(event.hostId, false) + } + + private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = { + updateExecutorSummary(execId) { exec => + val newInfo = newExecutorSummary(exec.info, + isBlacklisted = blacklisted) + new ExecutorSummaryWrapper(newInfo) + } + } + + private def updateNodeBlackList(host: String, blacklisted: Boolean): Unit = { + // Implicitly (un)blacklist every executor associated with the node. + activeExecutorIds(host).foreach { exec => updateBlackListStatus(exec, blacklisted) } + } + + override def onJobStart(event: SparkListenerJobStart): Unit = time(event) { + // Compute (a potential underestimate of) the number of tasks that will be run by this job. + // This may be an underestimate because the job start event references all of the result + // stages' transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + val numTasks = { + val missingStages = event.stageInfos.filter(_.completionTime.isEmpty) + missingStages.map(_.numTasks).sum + } + + val lastStageInfo = event.stageInfos.sortBy(_.stageId).lastOption + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val jobGroup = Option(event.properties) + .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) } + + val info = newJobData(None, + jobId = event.jobId, + name = lastStageName, + submissionTime = Option(event.time).filter(_ >= 0).map(new Date(_)), + stageIds = event.stageIds, + jobGroup = jobGroup, + status = JobExecutionStatus.RUNNING, + numTasks = numTasks) + + val initialStages = event.stageInfos.map { stage => + // A new job submission may re-use an existing stage, so this code needs to do an update + // instead of just a write. + updateStageData(stage.stageId, stage.attemptId) { wrapper => + val jobIds = wrapper.jobIds ++ Set(event.jobId) + val newStage = newStageData( + wrapper.info, + name = stage.name, + numTasks = stage.numTasks, + rddIds = stage.rddInfos.map(_.id)) + newStageDataWrapper(wrapper, newStage, jobIds = jobIds) + } + new JobStageSummary(stage.stageId, stage.attemptId, stage.numTasks) + } + + val stored = newJobDataWrapper(None, info, initialStages = initialStages) + kvstore.write(stored) + + activeJobs += event.jobId + + // Create the graph data for all the job's stages. + event.stageInfos.foreach { stage => + val graph = RDDOperationGraph.makeOperationGraph(stage, Int.MaxValue) + val uigraph = new RDDOperationGraphWrapper( + stage.stageId, + graph.edges, + graph.outgoingEdges, + graph.incomingEdges, + newRDDOperationCluster(graph.rootCluster)) + kvstore.write(uigraph) + } + } + + private def newRDDOperationCluster(cluster: RDDOperationCluster): RDDOperationClusterWrapper = { + new RDDOperationClusterWrapper( + cluster.id, + cluster.name, + cluster.childNodes, + cluster.childClusters.map(newRDDOperationCluster)) + } + + override def onJobEnd(event: SparkListenerJobEnd): Unit = time(event) { + updateJobData(event.jobId) { job => + val status = event.jobResult match { + case JobSucceeded => JobExecutionStatus.SUCCEEDED + case JobFailed(_) => JobExecutionStatus.FAILED + } + + val skippedStageIds = job.info.stageIds.toSet &~ job.submittedStages + val skippedStages = job.initialStages.filter { s => skippedStageIds.contains(s.stageId) } + + skippedStages.foreach { s => + updateStageData(s.stageId, s.attemptId) { stage => + val newInfo = newStageData(stage.info, status = v1.StageStatus.SKIPPED) + newStageDataWrapper(stage, newInfo) + } + } + + val newInfo = newJobData(job.info, + jobId = event.jobId, + completionTime = if (event.time != -1) Some(new Date(event.time)) else None, + status = status, + numSkippedStages = skippedStages.size, + numSkippedTasks = skippedStages.map(_.numTasks).sum) + newJobDataWrapper(job, newInfo) + } + + activeJobs -= event.jobId + } + + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = time(event) { + // New attempts are created based on the previous attempt's data - mainly to track which jobs + // depend on the stage. Attempt IDs start at 0, so this check takes care of retrieving the + // correct previous attempt from the store. + val previousAttempt = if (event.stageInfo.attemptId > 0) { + event.stageInfo.attemptId - 1 + } else { + event.stageInfo.attemptId + } + + val poolName = Option(event.properties).flatMap { p => + Option(p.getProperty("spark.scheduler.pool")) + }.getOrElse(SparkUI.DEFAULT_POOL_NAME) + + updateStageData(event.stageInfo.stageId, previousAttempt) { stage => + val description = Option(event.properties).flatMap { p => + Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) + } + + // Update the jobs related to the stage. + // - update the stage counts + // - remove the stage from the job's completed list + // - add the stage to the list of submitted stages for the stage, so that skipped stages + // are correctly counted. + activeJobs(stage).foreach { job => + updateJobData(job) { wrapper => + val completedStages = wrapper.completedStages - event.stageInfo.stageId + val submittedStages = wrapper.submittedStages + event.stageInfo.stageId + val newJobInfo = newJobData( + wrapper.info, + jobId = job, + numActiveStages = wrapper.info.numActiveStages + 1, + numCompletedStages = completedStages.size) + newJobDataWrapper(wrapper, newJobInfo, + submittedStages = submittedStages, + completedStages = completedStages) + } + } + + // SPARK-20205. The DAGScheduler posts the event before setting "submissionTime", so wait a + // bit if it's not set. Remove this when the scheduler code is fixed. + var retries = 10 + while (!event.stageInfo.submissionTime.isDefined && retries > 0) { + Thread.sleep(1) + retries -= 1 + } + + val newInfo = newStageData( + stage.info, + status = v1.StageStatus.ACTIVE, + stageId = event.stageInfo.stageId, + attemptId = event.stageInfo.attemptId, + numTasks = event.stageInfo.numTasks, + submissionTime = event.stageInfo.submissionTime.map(new Date(_)), + name = event.stageInfo.name, + description = description, + details = event.stageInfo.details, + schedulingPool = poolName) + newStageDataWrapper(stage, newInfo) + } + + updatePoolData(poolName) { pool => + new PoolData(pool.name, pool.stageIds + event.stageInfo.stageId) + } + + // Update RDDs with the stage name. + event.stageInfo.rddInfos.foreach { info => + updateRDDStorageInfo(info.id) { rdd => + val updated = newRDDStorageInfo(rdd.info, + name = info.name, + numPartitions = info.numPartitions, + storageLevel = info.storageLevel.description) + new RDDStorageInfoWrapper(updated) + } + } + } + + override def onTaskStart(event: SparkListenerTaskStart): Unit = time(event) { + if (event.taskInfo != null) { + val task = newTaskData(None, + taskId = event.taskInfo.taskId, + index = event.taskInfo.index, + attempt = event.taskInfo.attemptNumber, + launchTime = new Date(event.taskInfo.launchTime), + executorId = event.taskInfo.executorId, + host = event.taskInfo.host, + status = event.taskInfo.status, + taskLocality = event.taskInfo.taskLocality.toString(), + speculative = event.taskInfo.speculative) + kvstore.write(new TaskDataWrapper(task, event.stageId, event.stageAttemptId)) + } + + updateStageData(event.stageId, event.stageAttemptId) { stage => + // Update jobs to add active stage and remove the stage from the completed list. + activeJobs(stage).foreach { job => + updateJobData(job) { wrapper => + val newJobInfo = newJobData( + wrapper.info, + jobId = job, + numActiveTasks = wrapper.info.numActiveTasks + 1) + newJobDataWrapper(wrapper, newJobInfo) + } + } + + val firstTask = stage.info.firstTaskLaunchedTime.getOrElse( + new Date(event.taskInfo.launchTime)) + + val newInfo = newStageData( + stage.info, + stageId = event.stageId, + attemptId = event.stageAttemptId, + firstTaskLaunchedTime = Some(firstTask), + numActiveTasks = stage.info.numActiveTasks + 1) + newStageDataWrapper(stage, newInfo) + } + + updateExecutorSummary(event.taskInfo.executorId) { exec => + val updated = newExecutorSummary( + exec.info, + activeTasks = exec.info.activeTasks + 1, + totalTasks = exec.info.totalTasks + 1) + new ExecutorSummaryWrapper(updated) + } + } + + override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = time(event) { + updateTaskData(event.taskInfo.taskId) { task => + val newInfo = newTaskData( + task.info, + status = event.taskInfo.status, + resultFetchStart = Some(new Date(event.taskInfo.gettingResultTime))) + new TaskDataWrapper(newInfo, task.stageId, task.stageAttemptId) + } + } + + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = time(event) { + val info = event.taskInfo + // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task + // completion event is for. Let's just drop it here. This means we might have some speculation + // tasks on the web ui that are never marked as complete. + if (info == null || event.stageAttemptId == -1) { + return + } + + var metricsDelta: v1.TaskMetrics = null + + val executorRunTime = if (event.taskMetrics != null) { + event.taskMetrics.executorRunTime + } else { + event.taskInfo.duration + } + + updateTaskData(event.taskInfo.taskId) { task => + val errorMessage = event.reason match { + case Success => + None + case k: TaskKilled => + Some(k.reason) + case e: ExceptionFailure => // Handle ExceptionFailure because we might have accumUpdates + Some(e.toErrorString) + case e: TaskFailedReason => // All other failure cases + Some(e.toErrorString) + } + + metricsDelta = calculateMetricsDelta(Option(event.taskMetrics), task.info.taskMetrics) + + val newInfo = newTaskData( + task.info, + taskId = event.taskInfo.taskId, + // Even though TaskInfo has a "duration", the UI has always shown the executor run time + // as the task's duration. + duration = Some(executorRunTime), + status = event.taskInfo.status, + accumulatorUpdates = filterAccumulables(info.accumulables).toSeq.map(newAccumulableInfo), + errorMessage = errorMessage, + taskMetrics = Option(event.taskMetrics).map(newTaskMetrics)) + new TaskDataWrapper(newInfo, event.stageId, event.stageAttemptId) + } + + val (completedDelta, failedDelta, killedDelta) = event.reason match { + case Success => + (1, 0, 0) + case _: TaskKilled => + (0, 0, 1) + case _ => + (0, 1, 0) + } + + updateStageData(event.stageId, event.stageAttemptId) { stage => + val killReason = event.reason match { + case k: TaskKilled => Some(k.reason) + case _ => None + } + + val oldStage = stage.info + val newInfo = newStageData( + oldStage, + numActiveTasks = oldStage.numActiveTasks - 1, + numCompleteTasks = oldStage.numCompleteTasks + completedDelta, + numFailedTasks = oldStage.numFailedTasks + failedDelta, + numKilledTasks = oldStage.numKilledTasks + killedDelta, + executorRunTime = oldStage.executorRunTime + executorRunTime, + inputBytes = oldStage.inputBytes + metricsDelta.inputMetrics.bytesRead, + inputRecords = oldStage.inputRecords + metricsDelta.inputMetrics.recordsRead, + outputBytes = oldStage.outputBytes + metricsDelta.outputMetrics.bytesWritten, + outputRecords = oldStage.outputRecords + metricsDelta.outputMetrics.recordsWritten, + shuffleReadBytes = oldStage.shuffleReadBytes + + metricsDelta.shuffleReadMetrics.localBytesRead + + metricsDelta.shuffleReadMetrics.remoteBytesRead, + shuffleReadRecords = oldStage.shuffleReadRecords + + metricsDelta.shuffleReadMetrics.recordsRead, + shuffleWriteBytes = oldStage.shuffleWriteBytes + + metricsDelta.shuffleWriteMetrics.bytesWritten, + shuffleWriteRecords = oldStage.shuffleWriteRecords + + metricsDelta.shuffleWriteMetrics.recordsWritten, + memoryBytesSpilled = oldStage.memoryBytesSpilled + metricsDelta.memoryBytesSpilled, + diskBytesSpilled = oldStage.diskBytesSpilled + metricsDelta.diskBytesSpilled, + accumulatorUpdates = mergeAccumulables(oldStage.accumulatorUpdates, info.accumulables), + killedTasksSummary = killedTasksSummary(event.reason, oldStage.killedTasksSummary)) + + activeJobs(stage).foreach { job => + updateJobData(job) { wrapper => + val newJobInfo = newJobData( + wrapper.info, + jobId = job, + numActiveTasks = wrapper.info.numActiveTasks - 1, + numCompletedTasks = wrapper.info.numCompletedTasks + completedDelta, + numFailedTasks = wrapper.info.numFailedTasks + failedDelta, + numKilledTasks = wrapper.info.numKilledTasks + killedDelta, + killedTasksSummary = killedTasksSummary(event.reason, wrapper.info.killedTasksSummary)) + newJobDataWrapper(wrapper, newJobInfo) + } + } + + newStageDataWrapper(stage, newInfo) + } + + updateExecutorStageSummary( + event.stageId, + event.stageAttemptId, + event.taskInfo.executorId) { exec => + newExecutorStageSummary( + exec, + taskTime = exec.taskTime + info.duration, + failedTasks = exec.failedTasks + failedDelta, + succeededTasks = exec.succeededTasks + completedDelta, + inputBytes = exec.inputBytes + metricsDelta.inputMetrics.bytesRead, + outputBytes = exec.outputBytes + metricsDelta.outputMetrics.bytesWritten, + shuffleRead = exec.shuffleRead + metricsDelta.shuffleReadMetrics.localBytesRead + + metricsDelta.shuffleReadMetrics.remoteBytesRead, + shuffleReadRecords = exec.shuffleReadRecords + metricsDelta.shuffleReadMetrics.recordsRead, + shuffleWrite = exec.shuffleWrite + metricsDelta.shuffleWriteMetrics.bytesWritten, + shuffleWriteRecords = exec.shuffleWriteRecords + + metricsDelta.shuffleWriteMetrics.recordsWritten, + memoryBytesSpilled = exec.memoryBytesSpilled + metricsDelta.memoryBytesSpilled, + diskBytesSpilled = exec.diskBytesSpilled + metricsDelta.diskBytesSpilled) + } + + updateExecutorSummary(event.taskInfo.executorId) { exec => + val (gcTime, inputBytes, shuffleRead, shuffleWrite) = if (event.taskMetrics != null) { + val readMetrics = event.taskMetrics.shuffleReadMetrics + (event.taskMetrics.jvmGCTime, + event.taskMetrics.inputMetrics.bytesRead, + readMetrics.localBytesRead + readMetrics.remoteBytesRead, + event.taskMetrics.shuffleWriteMetrics.bytesWritten) + } else { + (0L, 0L, 0L, 0L) + } + + val updated = newExecutorSummary( + exec.info, + activeTasks = exec.info.activeTasks - 1, + completedTasks = exec.info.completedTasks + completedDelta, + failedTasks = exec.info.failedTasks + failedDelta, + totalDuration = exec.info.totalDuration + event.taskInfo.duration, + totalGCTime = exec.info.totalGCTime + gcTime, + totalInputBytes = exec.info.totalInputBytes + inputBytes, + totalShuffleRead = exec.info.totalShuffleRead + shuffleRead, + totalShuffleWrite = exec.info.totalShuffleWrite + shuffleWrite) + new ExecutorSummaryWrapper(updated) + } + } + + override def onStageCompleted(event: SparkListenerStageCompleted): Unit = time(event) { + val info = event.stageInfo + updateStageData(info.stageId, info.attemptId) { stage => + val status = event.stageInfo.failureReason match { + case Some(_) => v1.StageStatus.FAILED + case None => v1.StageStatus.COMPLETE + } + + // For some reason this data is filled in the "completed" event, not in the "submitted" + // event. Go figure. + val submissionTime = event.stageInfo.submissionTime.map(new Date(_)).orElse( + stage.info.submissionTime) + val accumulables = filterAccumulables(info.accumulables.values).map(newAccumulableInfo).toSeq + val newStage = newStageData( + stage.info, + status = status, + submissionTime = submissionTime, + completionTime = info.completionTime.map(new Date(_)), + failureReason = info.failureReason, + accumulatorUpdates = accumulables) + + val (completedDelta, failedDelta) = if (status == v1.StageStatus.COMPLETE) { + (1, 0) + } else { + (0, 1) + } + + activeJobs(stage).foreach { job => + updateJobData(job) { wrapper => + val stages = if (status == v1.StageStatus.COMPLETE) { + wrapper.completedStages + info.stageId + } else { + wrapper.completedStages + } + val newJobInfo = newJobData( + wrapper.info, + jobId = job, + numActiveStages = wrapper.info.numActiveStages - 1, + numCompletedStages = stages.size, + numFailedStages = wrapper.info.numFailedStages + failedDelta) + newJobDataWrapper(wrapper, newJobInfo, completedStages = stages) + } + } + + updatePoolData(newStage.schedulingPool) { pool => + new PoolData(pool.name, pool.stageIds - newStage.stageId) + } + + newStageDataWrapper(stage, newStage) + } + } + + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = time(event) { + // This needs to set fields that are already set by onExecutorAdded because the driver is + // considered an "executor" in the UI, but does not have a SparkListenerExecutorAdded event. + updateExecutorSummary(event.blockManagerId.executorId) { exec => + // Only create memory metrics if the event has the info; this avoid setting bogus values + // when replaying old application logs. + val memMetrics = event.maxOnHeapMem.map { _ => + newMemoryMetrics( + exec.info.memoryMetrics, + totalOnHeapStorageMemory = event.maxOnHeapMem, + totalOffHeapStorageMemory = event.maxOffHeapMem) + } + + val updated = newExecutorSummary( + exec.info, + hostPort = event.blockManagerId.hostPort, + isActive = true, + maxMemory = event.maxMem, + memoryMetrics = memMetrics) + val newWrapper = new ExecutorSummaryWrapper(updated) + activeExecutorIds.getOrElseUpdate(newWrapper.host, new HashSet()) += newWrapper.id + newWrapper + } + } + + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { + // Nothing to do here. Covered by onExecutorRemoved. + } + + override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = time(event) { + kvstore.delete(classOf[RDDStorageInfoWrapper], event.rddId) + } + + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + time(event) { + event.accumUpdates.foreach { case (taskId, sid, sAttempt, accumUpdates) => + val task = read[TaskDataWrapper](taskId) + val metrics = TaskMetrics.fromAccumulatorInfos(accumUpdates) + val delta = calculateMetricsDelta(Some(metrics), task.flatMap(_.info.taskMetrics)) + + updateStageData(sid, sAttempt) { stage => + val oldStage = stage.info + + // Note that this update does not update the stage's accumulator list. The updates + // received in the event are partial and only refer to a specific executor, so there + // isn't enough information to update the non-internal accumulators with their current + // values. + val updatedStage = newStageData( + oldStage, + inputBytes = oldStage.inputBytes + delta.inputMetrics.bytesRead, + inputRecords = oldStage.inputRecords + delta.inputMetrics.recordsRead, + outputBytes = oldStage.outputBytes + delta.outputMetrics.bytesWritten, + outputRecords = oldStage.outputRecords + delta.outputMetrics.recordsWritten, + shuffleReadBytes = oldStage.shuffleReadBytes + + delta.shuffleReadMetrics.localBytesRead + + delta.shuffleReadMetrics.remoteBytesRead, + shuffleReadRecords = oldStage.shuffleReadRecords + + delta.shuffleReadMetrics.recordsRead, + shuffleWriteBytes = oldStage.shuffleWriteBytes + + delta.shuffleWriteMetrics.bytesWritten, + shuffleWriteRecords = oldStage.shuffleWriteRecords + + delta.shuffleWriteMetrics.recordsWritten, + memoryBytesSpilled = oldStage.memoryBytesSpilled + delta.memoryBytesSpilled, + diskBytesSpilled = oldStage.diskBytesSpilled + delta.diskBytesSpilled) + newStageDataWrapper(stage, updatedStage) + } + + updateExecutorStageSummary(sid, sAttempt, event.execId) { exec => + newExecutorStageSummary( + exec, + inputBytes = exec.inputBytes + delta.inputMetrics.bytesRead, + outputBytes = exec.outputBytes + delta.outputMetrics.bytesWritten, + shuffleRead = exec.shuffleRead + delta.shuffleReadMetrics.localBytesRead + + delta.shuffleReadMetrics.remoteBytesRead, + shuffleReadRecords = exec.shuffleReadRecords + delta.shuffleReadMetrics.recordsRead, + shuffleWrite = exec.shuffleWrite + delta.shuffleWriteMetrics.bytesWritten, + shuffleWriteRecords = exec.shuffleWriteRecords + + delta.shuffleWriteMetrics.recordsWritten, + memoryBytesSpilled = exec.memoryBytesSpilled + delta.memoryBytesSpilled, + diskBytesSpilled = exec.diskBytesSpilled + delta.diskBytesSpilled) + } + } + } + } + + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = time(event) { + event.blockUpdatedInfo.blockId match { + case block: RDDBlockId => updateRDDBlock(event, block) + case stream: StreamBlockId => updateStreamBlock(event, stream) + case _ => + } + } + + private def updateRDDBlock(event: SparkListenerBlockUpdated, block: RDDBlockId): Unit = { + val executorId = event.blockUpdatedInfo.blockManagerId.executorId + + // Whether values are being added to or removed from the existing accounting. + val storageLevel = event.blockUpdatedInfo.storageLevel + val diskMult = if (storageLevel.useDisk) 1 else -1 + val memoryMult = if (storageLevel.useMemory) 1 else -1 + + // Function to apply a delta to a value, but ensure that it doesn't go negative. + def newValue(old: Long, delta: Long): Long = math.max(0, old + delta) + + // Function to calculate the remaining memory after applying a delta. Assumes that if + // one value is provided, all the values are. + def remainingMemory(max: Option[Long], oldv: Option[Long], newv: Option[Long]): Option[Long] = { + max.map { m => newValue(m, oldv.get - newv.get) } + } + + // If the storage level is NONE, then don't update the storage level of existing information. + val updatedStorageLevel = if (storageLevel.useMemory || storageLevel.useDisk) { + Some(storageLevel.description) + } else { + None + } + + // We need information about the executor to update some memory accounting values in the + // RDD info, so read that beforehand. + val executorInfo = read[ExecutorSummaryWrapper](executorId).getOrElse( + new ExecutorSummaryWrapper(newExecutorSummary(None, id = executorId))) + + var rddBlocksDelta = 0 + val memMetrics = executorInfo.info.memoryMetrics + + // Update the block entry in the RDD info, keeping track of the deltas above so that we + // can update the executor information too. + updateRDDStorageInfo(block.rddId) { rdd => + val (maybeOldBlock, others) = rdd.info.partitions.getOrElse(Nil) + .partition(_.blockName == block.name) match { + case (old, others) => + (old.headOption, others) + } + + val oldBlock = maybeOldBlock.getOrElse { + newRDDPartitionInfo(None, blockName = block.name) + } + + val newExecutors = if (storageLevel.useDisk || storageLevel.useMemory) { + if (!oldBlock.executors.contains(executorId)) { + rddBlocksDelta = 1 + } + oldBlock.executors.toSet + executorId + } else { + rddBlocksDelta = -1 + oldBlock.executors.toSet - executorId + } + + // Only update the block if it's still stored in some executor, otherwise get rid of it. + val newBlocks = if (newExecutors.nonEmpty) { + val newBlock = newRDDPartitionInfo( + oldBlock, + storageLevel = updatedStorageLevel, + memoryUsed = newValue(oldBlock.memoryUsed, event.blockUpdatedInfo.memSize * memoryMult), + diskUsed = newValue(oldBlock.diskUsed, event.blockUpdatedInfo.diskSize * diskMult), + executors = newExecutors.toSeq) + + Seq(newBlock) + } else { + Nil + } + + val address = event.blockUpdatedInfo.blockManagerId.hostPort + val (oldDist, otherDists) = rdd.info.dataDistribution.getOrElse(Nil) + .partition(_.address == address) match { + case (old, others) => + val _old = old.headOption.getOrElse { + newRDDDataDistribution( + None, + address = address, + memoryRemaining = executorInfo.info.maxMemory, + onHeapMemoryUsed = memMetrics.map { _ => 0L }, + offHeapMemoryUsed = memMetrics.map { _ => 0L }, + onHeapMemoryRemaining = memMetrics.map(_.totalOnHeapStorageMemory), + offHeapMemoryRemaining = memMetrics.map(_.totalOffHeapStorageMemory)) + } + (_old, others) + } + + // If the new distribution is empty, just do not add it to the new RDD info. + val newDistMem = newValue(oldDist.memoryUsed, event.blockUpdatedInfo.memSize * memoryMult) + val newDistDisk = newValue(oldDist.diskUsed, event.blockUpdatedInfo.diskSize * diskMult) + val newDists = if (newDistMem > 0 || newDistDisk > 0) { + val _newMem = Some(newDistMem) + val (newOffHeap, remainingOffHeap) = if (storageLevel.useOffHeap) { + (_newMem, remainingMemory(oldDist.offHeapMemoryRemaining, oldDist.offHeapMemoryUsed, + _newMem)) + } else { + (oldDist.offHeapMemoryUsed, oldDist.offHeapMemoryRemaining) + } + val (newOnHeap, remainingOnHeap) = if (!storageLevel.useOffHeap) { + (_newMem, remainingMemory(oldDist.onHeapMemoryRemaining, oldDist.onHeapMemoryUsed, + _newMem)) + } else { + (oldDist.onHeapMemoryUsed, oldDist.onHeapMemoryRemaining) + } + + val newDist = newRDDDataDistribution( + oldDist, + memoryUsed = newDistMem, + memoryRemaining = newValue(oldDist.memoryRemaining, + event.blockUpdatedInfo.memSize * memoryMult * -1), + diskUsed = newDistDisk, + onHeapMemoryUsed = newOnHeap, + offHeapMemoryUsed = newOffHeap, + onHeapMemoryRemaining = remainingOnHeap, + offHeapMemoryRemaining = remainingOffHeap) + Seq(newDist) + } else { + Nil + } + + val allBlocks = others ++ newBlocks + val newRDD = newRDDStorageInfo( + rdd.info, + numCachedPartitions = allBlocks.size, + storageLevel = updatedStorageLevel, + memoryUsed = newValue(rdd.info.memoryUsed, event.blockUpdatedInfo.memSize * memoryMult), + diskUsed = newValue(rdd.info.diskUsed, event.blockUpdatedInfo.diskSize * diskMult), + dataDistribution = Some(otherDists ++ newDists), + partitions = Some(allBlocks)) + new RDDStorageInfoWrapper(newRDD) + } + + // Update the ExecutorSummary for the block's manager. + val updatedMemMetrics = memMetrics.map { m => + val onHeapUsed = if (!storageLevel.useOffHeap) { + newValue(m.usedOnHeapStorageMemory, event.blockUpdatedInfo.memSize * memoryMult) + } else { + m.usedOnHeapStorageMemory + } + val offHeapUsed = if (storageLevel.useOffHeap) { + newValue(m.usedOffHeapStorageMemory, event.blockUpdatedInfo.memSize * memoryMult) + } else { + m.usedOffHeapStorageMemory + } + newMemoryMetrics(m, + usedOnHeapStorageMemory = onHeapUsed, + usedOffHeapStorageMemory = offHeapUsed) + } + + val newExecSummary = newExecutorSummary( + executorInfo.info, + rddBlocks = newValue(executorInfo.info.rddBlocks, rddBlocksDelta).toInt, + memoryUsed = newValue(executorInfo.info.memoryUsed, + event.blockUpdatedInfo.memSize * memoryMult), + diskUsed = newValue(executorInfo.info.diskUsed, + event.blockUpdatedInfo.diskSize * diskMult), + memoryMetrics = updatedMemMetrics) + kvstore.write(new ExecutorSummaryWrapper(newExecSummary)) + } + + private def updateStreamBlock(event: SparkListenerBlockUpdated, stream: StreamBlockId): Unit = { + val storageLevel = event.blockUpdatedInfo.storageLevel + if (storageLevel.isValid) { + val data = new StreamBlockData( + stream.name, + event.blockUpdatedInfo.blockManagerId.executorId, + event.blockUpdatedInfo.blockManagerId.hostPort, + storageLevel.description, + storageLevel.useMemory, + storageLevel.useDisk, + storageLevel.deserialized, + event.blockUpdatedInfo.memSize, + event.blockUpdatedInfo.diskSize) + kvstore.write(data) + } else { + kvstore.delete(classOf[StreamBlockData], + Array(stream.name, event.blockUpdatedInfo.blockManagerId.executorId)) + } + } + + private def updateJobData(id: Int)(fn: JobDataWrapper => JobDataWrapper): Unit = { + update[JobDataWrapper](id) { old => + val job = old.getOrElse((newJobDataWrapper(None, newJobData(None)))) + fn(job) + } + } + + private def updateStageData( + stageId: Int, + attemptId: Int) + (fn: StageDataWrapper => StageDataWrapper): Unit = { + update[StageDataWrapper](Array(stageId, attemptId)) { old => + val stage = old.getOrElse { + newStageDataWrapper(None, newStageData(None, stageId = stageId, attemptId = attemptId)) + } + fn(stage) + } + } + + private def updateTaskData(id: Long)(fn: TaskDataWrapper => TaskDataWrapper): Unit = { + update[TaskDataWrapper](id) { old => + val task = old.getOrElse(new TaskDataWrapper(newTaskData(None, taskId = id), -1, -1)) + fn(task) + } + } + + private def updateExecutorSummary( + id: String) + (fn: ExecutorSummaryWrapper => ExecutorSummaryWrapper): Unit = { + update[ExecutorSummaryWrapper](id) { old => + val exec = old.getOrElse( + new ExecutorSummaryWrapper(newExecutorSummary(None, id = id))) + fn(exec) + } + } + + private def updateRDDStorageInfo( + id: Int) + (fn: RDDStorageInfoWrapper => RDDStorageInfoWrapper): Unit = { + update[RDDStorageInfoWrapper](id) { old => + val rdd = old.getOrElse(new RDDStorageInfoWrapper(newRDDStorageInfo(None, id = id))) + fn(rdd) + } + } + + private def updateExecutorStageSummary( + stageId: Int, + stageAttemptId: Int, + executorId: String) + (fn: v1.ExecutorStageSummary => v1.ExecutorStageSummary): Unit = { + update[ExecutorStageSummaryWrapper](Array(stageId, stageAttemptId, executorId)) { old => + val oldInfo = old.map(_.info).getOrElse(newExecutorStageSummary(None)) + new ExecutorStageSummaryWrapper(stageId, stageAttemptId, executorId, fn(oldInfo)) + } + } + + private def updatePoolData(name: String)(fn: PoolData => PoolData): Unit = { + update[PoolData](name) { old => + fn(old.getOrElse(new PoolData(name, Set()))) + } + } + + /** + * Return a new TaskMetrics object containing the delta of the various fields of the given + * metrics objects. This is currently targeted at updating stage data, so it does not + * necessarily calculate deltas for all the fields. + */ + private def calculateMetricsDelta( + taskMetrics: Option[TaskMetrics], + oldMetrics: Option[v1.TaskMetrics]): v1.TaskMetrics = { + + val metrics = taskMetrics.getOrElse(TaskMetrics.empty) + val old = oldMetrics.getOrElse(newTaskMetrics(TaskMetrics.empty)) + + val shuffleWriteDelta = new v1.ShuffleWriteMetrics( + metrics.shuffleWriteMetrics.bytesWritten - old.shuffleWriteMetrics.bytesWritten, + 0L, + metrics.shuffleWriteMetrics.recordsWritten - old.shuffleWriteMetrics.recordsWritten) + + val shuffleReadDelta = new v1.ShuffleReadMetrics( + 0L, 0L, 0L, + metrics.shuffleReadMetrics.remoteBytesRead - old.shuffleReadMetrics.remoteBytesRead, + metrics.shuffleReadMetrics.localBytesRead - old.shuffleReadMetrics.localBytesRead, + metrics.shuffleReadMetrics.recordsRead - old.shuffleReadMetrics.recordsRead) + + val inputDelta = new v1.InputMetrics( + metrics.inputMetrics.bytesRead - old.inputMetrics.bytesRead, + metrics.inputMetrics.recordsRead - old.inputMetrics.recordsRead) + + val outputDelta = new v1.OutputMetrics( + metrics.outputMetrics.bytesWritten - old.outputMetrics.bytesWritten, + metrics.outputMetrics.recordsWritten - old.outputMetrics.recordsWritten) + + new v1.TaskMetrics( + 0L, 0L, + metrics.executorRunTime - old.executorRunTime, + metrics.executorCpuTime - old.executorCpuTime, + 0L, 0L, 0L, + metrics.memoryBytesSpilled - old.memoryBytesSpilled, + metrics.diskBytesSpilled - old.diskBytesSpilled, + 0L, + inputDelta, + outputDelta, + shuffleReadDelta, + shuffleWriteDelta) + } + // scalastyle:off argCount + + // The following are "copy methods" for API types that are often modified by the event handlers + // above. They allow copying from an existing instance, overriding specific fields with the + // values to be updated. Since the API types are immutable, this makes the event handlers cleaner + // since they don't have to deal with default and existing values. + + private def newExecutorSummary( + old: Option[v1.ExecutorSummary], + id: Option[String] = None, + hostPort: Option[String] = None, + isActive: Option[Boolean] = None, + rddBlocks: Option[Int] = None, + memoryUsed: Option[Long] = None, + diskUsed: Option[Long] = None, + totalCores: Option[Int] = None, + maxTasks: Option[Int] = None, + activeTasks: Option[Int] = None, + failedTasks: Option[Int] = None, + completedTasks: Option[Int] = None, + totalTasks: Option[Int] = None, + totalDuration: Option[Long] = None, + totalGCTime: Option[Long] = None, + totalInputBytes: Option[Long] = None, + totalShuffleRead: Option[Long] = None, + totalShuffleWrite: Option[Long] = None, + isBlacklisted: Option[Boolean] = None, + maxMemory: Option[Long] = None, + executorLogs: Option[Map[String, String]] = None, + memoryMetrics: Option[Option[v1.MemoryMetrics]] = None) : v1.ExecutorSummary = { + new v1.ExecutorSummary( + value(id, old.map(_.id), UNKNOWN), + value(hostPort, old.map(_.hostPort), UNKNOWN), + value(isActive, old.map(_.isActive), false), + value(rddBlocks, old.map(_.rddBlocks), 0), + value(memoryUsed, old.map(_.memoryUsed), 0L), + value(diskUsed, old.map(_.diskUsed), 0L), + value(totalCores, old.map(_.totalCores), 0), + value(maxTasks, old.map(_.maxTasks), 0), + value(activeTasks, old.map(_.activeTasks), 0), + value(failedTasks, old.map(_.failedTasks), 0), + value(completedTasks, old.map(_.completedTasks), 0), + value(totalTasks, old.map(_.totalTasks), 0), + value(totalDuration, old.map(_.totalDuration), 0L), + value(totalGCTime, old.map(_.totalGCTime), 0L), + value(totalInputBytes, old.map(_.totalInputBytes), 0L), + value(totalShuffleRead, old.map(_.totalShuffleRead), 0L), + value(totalShuffleWrite, old.map(_.totalShuffleWrite), 0L), + value(isBlacklisted, old.map(_.isBlacklisted), false), + value(maxMemory, old.map(_.maxMemory), 0L), + value(executorLogs, old.map(_.executorLogs), Map()), + option(memoryMetrics, old.map(_.memoryMetrics))) + } + + private def newMemoryMetrics( + old: Option[v1.MemoryMetrics], + usedOnHeapStorageMemory: Option[Long] = None, + usedOffHeapStorageMemory: Option[Long] = None, + totalOnHeapStorageMemory: Option[Long] = None, + totalOffHeapStorageMemory: Option[Long] = None): v1.MemoryMetrics = { + new v1.MemoryMetrics( + value(usedOnHeapStorageMemory, old.map(_.usedOnHeapStorageMemory), 0L), + value(usedOffHeapStorageMemory, old.map(_.usedOffHeapStorageMemory), 0L), + value(totalOnHeapStorageMemory, old.map(_.totalOnHeapStorageMemory), 0L), + value(totalOffHeapStorageMemory, old.map(_.totalOffHeapStorageMemory), 0L)) + } + + private def newJobData( + old: Option[v1.JobData], + jobId: Option[Int] = None, + name: Option[String] = None, + description: Option[Option[String]] = None, + submissionTime: Option[Option[Date]] = None, + completionTime: Option[Option[Date]] = None, + stageIds: Option[Seq[Int]] = None, + jobGroup: Option[Option[String]] = None, + status: Option[JobExecutionStatus] = None, + numTasks: Option[Int] = None, + numActiveTasks: Option[Int] = None, + numCompletedTasks: Option[Int] = None, + numSkippedTasks: Option[Int] = None, + numFailedTasks: Option[Int] = None, + numKilledTasks: Option[Int] = None, + numActiveStages: Option[Int] = None, + numCompletedStages: Option[Int] = None, + numSkippedStages: Option[Int] = None, + numFailedStages: Option[Int] = None, + killedTasksSummary: Option[Map[String, Int]] = None): v1.JobData = { + new v1.JobData( + value(jobId, old.map(_.jobId), -1), + value(name, old.map(_.name), null), + option(description, old.map(_.description)), + option(submissionTime, old.map(_.submissionTime)), + option(completionTime, old.map(_.completionTime)), + value(stageIds, old.map(_.stageIds), Nil), + option(jobGroup, old.map(_.jobGroup)), + value(status, old.map(_.status), null), + value(numTasks, old.map(_.numTasks), 0), + value(numActiveTasks, old.map(_.numActiveTasks), 0), + value(numCompletedTasks, old.map(_.numCompletedTasks), 0), + value(numSkippedTasks, old.map(_.numSkippedTasks), 0), + value(numFailedTasks, old.map(_.numFailedTasks), 0), + value(numKilledTasks, old.map(_.numKilledTasks), 0), + value(numActiveStages, old.map(_.numActiveStages), 0), + value(numCompletedStages, old.map(_.numCompletedStages), 0), + value(numSkippedStages, old.map(_.numSkippedStages), 0), + value(numFailedStages, old.map(_.numFailedStages), 0), + value(killedTasksSummary, old.map(_.killedTasksSummary), Map())) + } + + private def newStageData( + old: Option[v1.StageData], + status: Option[v1.StageStatus] = None, + stageId: Option[Int] = None, + attemptId: Option[Int] = None, + numTasks: Option[Int] = None, + numActiveTasks: Option[Int] = None, + numCompleteTasks: Option[Int] = None, + numFailedTasks: Option[Int] = None, + numKilledTasks: Option[Int] = None, + executorRunTime: Option[Long] = None, + executorCpuTime: Option[Long] = None, + submissionTime: Option[Option[Date]] = None, + firstTaskLaunchedTime: Option[Option[Date]] = None, + completionTime: Option[Option[Date]] = None, + failureReason: Option[Option[String]] = None, + inputBytes: Option[Long] = None, + inputRecords: Option[Long] = None, + outputBytes: Option[Long] = None, + outputRecords: Option[Long] = None, + shuffleReadBytes: Option[Long] = None, + shuffleReadRecords: Option[Long] = None, + shuffleWriteBytes: Option[Long] = None, + shuffleWriteRecords: Option[Long] = None, + memoryBytesSpilled: Option[Long] = None, + diskBytesSpilled: Option[Long] = None, + name: Option[String] = None, + description: Option[Option[String]] = None, + details: Option[String] = None, + schedulingPool: Option[String] = None, + rddIds: Option[Seq[Int]] = None, + accumulatorUpdates: Option[Seq[v1.AccumulableInfo]] = None, + killedTasksSummary: Option[Map[String, Int]] = None) + : v1.StageData = { + new v1.StageData( + value(status, old.map(_.status), v1.StageStatus.PENDING), + value(stageId, old.map(_.stageId), -1), + value(attemptId, old.map(_.attemptId), -1), + value(numTasks, old.map(_.numTasks), 0), + value(numActiveTasks, old.map(_.numActiveTasks), 0), + value(numCompleteTasks, old.map(_.numCompleteTasks), 0), + value(numFailedTasks, old.map(_.numFailedTasks), 0), + value(numKilledTasks, old.map(_.numKilledTasks), 0), + value(executorRunTime, old.map(_.executorRunTime), 0L), + value(executorCpuTime, old.map(_.executorCpuTime), 0L), + option(submissionTime, old.map(_.submissionTime)), + option(firstTaskLaunchedTime, old.map(_.firstTaskLaunchedTime)), + option(completionTime, old.map(_.completionTime)), + option(failureReason, old.map(_.failureReason)), + value(inputBytes, old.map(_.inputBytes), 0L), + value(inputRecords, old.map(_.inputRecords), 0L), + value(outputBytes, old.map(_.outputBytes), 0L), + value(outputRecords, old.map(_.outputRecords), 0L), + value(shuffleReadBytes, old.map(_.shuffleReadBytes), 0L), + value(shuffleReadRecords, old.map(_.shuffleReadRecords), 0L), + value(shuffleWriteBytes, old.map(_.shuffleWriteBytes), 0L), + value(shuffleWriteRecords, old.map(_.shuffleWriteRecords), 0L), + value(memoryBytesSpilled, old.map(_.memoryBytesSpilled), 0L), + value(diskBytesSpilled, old.map(_.diskBytesSpilled), 0L), + value(name, old.map(_.name), null), + option(description, old.map(_.description)), + value(details, old.map(_.details), null), + value(schedulingPool, old.map(_.schedulingPool), null), + value(rddIds, old.map(_.rddIds), Nil), + value(accumulatorUpdates, old.map(_.accumulatorUpdates), Nil), + None, // Task list is always empty; it's stored separately. + None, // Executor summary us always empty; it's stored separately. + value(killedTasksSummary, old.map(_.killedTasksSummary), Map())) + } + + private def newExecutorStageSummary( + old: Option[v1.ExecutorStageSummary], + taskTime: Option[Long] = None, + failedTasks: Option[Int] = None, + succeededTasks: Option[Int] = None, + killedTasks: Option[Int] = None, + inputBytes: Option[Long] = None, + inputRecords: Option[Long] = None, + outputBytes: Option[Long] = None, + outputRecords: Option[Long] = None, + shuffleRead: Option[Long] = None, + shuffleReadRecords: Option[Long] = None, + shuffleWrite: Option[Long] = None, + shuffleWriteRecords: Option[Long] = None, + memoryBytesSpilled: Option[Long] = None, + diskBytesSpilled: Option[Long] = None): v1.ExecutorStageSummary = { + new v1.ExecutorStageSummary( + value(taskTime, old.map(_.taskTime), 0L), + value(failedTasks, old.map(_.failedTasks), 0), + value(succeededTasks, old.map(_.succeededTasks), 0), + value(killedTasks, old.map(_.killedTasks), 0), + value(inputBytes, old.map(_.inputBytes), 0L), + value(inputRecords, old.map(_.inputRecords), 0L), + value(outputBytes, old.map(_.outputBytes), 0L), + value(outputRecords, old.map(_.outputRecords), 0L), + value(shuffleRead, old.map(_.shuffleRead), 0L), + value(shuffleReadRecords, old.map(_.shuffleReadRecords), 0L), + value(shuffleWrite, old.map(_.shuffleWrite), 0L), + value(shuffleWriteRecords, old.map(_.shuffleWriteRecords), 0L), + value(memoryBytesSpilled, old.map(_.memoryBytesSpilled), 0L), + value(diskBytesSpilled, old.map(_.diskBytesSpilled), 0L)) + } + + private def newTaskData( + old: Option[v1.TaskData], + taskId: Option[Long] = None, + index: Option[Int] = None, + attempt: Option[Int] = None, + launchTime: Option[Date] = None, + resultFetchStart: Option[Option[Date]] = None, + duration: Option[Option[Long]] = None, + executorId: Option[String] = None, + host: Option[String] = None, + status: Option[String] = None, + taskLocality: Option[String] = None, + speculative: Option[Boolean] = None, + accumulatorUpdates: Option[Seq[v1.AccumulableInfo]] = None, + errorMessage: Option[Option[String]] = None, + taskMetrics: Option[Option[v1.TaskMetrics]] = None): v1.TaskData = { + new v1.TaskData( + value(taskId, old.map(_.taskId), -1L), + value(index, old.map(_.index), -1), + value(attempt, old.map(_.attempt), -1), + value(launchTime, old.map(_.launchTime), DEFAULT_DATE), + option(resultFetchStart, old.map(_.resultFetchStart)), + option(duration, old.map(_.duration)), + value(executorId, old.map(_.executorId), UNKNOWN).intern(), + value(host, old.map(_.host), UNKNOWN).intern(), + value(status, old.map(_.status), UNKNOWN).intern(), + value(taskLocality, old.map(_.taskLocality), UNKNOWN).intern(), + value(speculative, old.map(_.speculative), false), + value(accumulatorUpdates, old.map(_.accumulatorUpdates), Nil), + option(errorMessage, old.map(_.errorMessage)), + option(taskMetrics, old.map(_.taskMetrics))) + } + + private def newRDDStorageInfo( + old: Option[v1.RDDStorageInfo], + id: Option[Int] = None, + name: Option[String] = None, + numPartitions: Option[Int] = None, + numCachedPartitions: Option[Int] = None, + storageLevel: Option[String] = None, + memoryUsed: Option[Long] = None, + diskUsed: Option[Long] = None, + dataDistribution: Option[Option[Seq[v1.RDDDataDistribution]]] = None, + partitions: Option[Option[Seq[v1.RDDPartitionInfo]]] = None): v1.RDDStorageInfo = { + new v1.RDDStorageInfo( + value(id, old.map(_.id), -1), + value(name, old.map(_.name), UNKNOWN), + value(numPartitions, old.map(_.numPartitions), 0), + value(numCachedPartitions, old.map(_.numCachedPartitions), 0), + value(storageLevel, old.map(_.storageLevel), StorageLevel.NONE.toString()), + value(memoryUsed, old.map(_.memoryUsed), 0L), + value(diskUsed, old.map(_.diskUsed), 0L), + option(dataDistribution, old.map(_.dataDistribution)), + option(partitions, old.map(_.partitions))) + } + + private def newRDDDataDistribution( + old: Option[v1.RDDDataDistribution], + address: Option[String] = None, + memoryUsed: Option[Long] = None, + memoryRemaining: Option[Long] = None, + diskUsed: Option[Long] = None, + onHeapMemoryUsed: Option[Option[Long]] = None, + offHeapMemoryUsed: Option[Option[Long]] = None, + onHeapMemoryRemaining: Option[Option[Long]] = None, + offHeapMemoryRemaining: Option[Option[Long]] = None): v1.RDDDataDistribution = { + new v1.RDDDataDistribution( + value(address, old.map(_.address), UNKNOWN), + value(memoryUsed, old.map(_.memoryUsed), 0L), + value(memoryRemaining, old.map(_.memoryRemaining), 0L), + value(diskUsed, old.map(_.diskUsed), 0L), + option(onHeapMemoryUsed, old.map(_.onHeapMemoryUsed)), + option(offHeapMemoryUsed, old.map(_.offHeapMemoryUsed)), + option(onHeapMemoryRemaining, old.map(_.onHeapMemoryRemaining)), + option(offHeapMemoryRemaining, old.map(_.offHeapMemoryRemaining))) + } + + private def newRDDPartitionInfo( + old: Option[v1.RDDPartitionInfo], + blockName: Option[String] = None, + storageLevel: Option[String] = None, + memoryUsed: Option[Long] = None, + diskUsed: Option[Long] = None, + executors: Option[Seq[String]] = None): v1.RDDPartitionInfo = { + new v1.RDDPartitionInfo( + value(blockName, old.map(_.blockName), UNKNOWN), + value(storageLevel, old.map(_.storageLevel), StorageLevel.NONE.toString()), + value(memoryUsed, old.map(_.memoryUsed), 0L), + value(diskUsed, old.map(_.diskUsed), 0L), + value(executors, old.map(_.executors), Nil)) + } + + // scalastyle:on argCount + + private def newAccumulableInfo(acc: AccumulableInfo): v1.AccumulableInfo = { + new v1.AccumulableInfo( + acc.id, + acc.name.map(_.intern()).orNull, + acc.update.map(_.toString()), + acc.value.map(_.toString()).orNull) + } + + private def newTaskMetrics(metrics: TaskMetrics): v1.TaskMetrics = { + new v1.TaskMetrics( + metrics.executorDeserializeTime, + metrics.executorDeserializeCpuTime, + metrics.executorRunTime, + metrics.executorCpuTime, + metrics.resultSize, + metrics.jvmGCTime, + metrics.resultSerializationTime, + metrics.memoryBytesSpilled, + metrics.diskBytesSpilled, + metrics.peakExecutionMemory, + new v1.InputMetrics( + metrics.inputMetrics.bytesRead, + metrics.inputMetrics.recordsRead), + new v1.OutputMetrics( + metrics.outputMetrics.bytesWritten, + metrics.outputMetrics.recordsWritten), + new v1.ShuffleReadMetrics( + metrics.shuffleReadMetrics.remoteBlocksFetched, + metrics.shuffleReadMetrics.localBlocksFetched, + metrics.shuffleReadMetrics.fetchWaitTime, + metrics.shuffleReadMetrics.remoteBytesRead, + metrics.shuffleReadMetrics.localBytesRead, + metrics.shuffleReadMetrics.recordsRead), + new v1.ShuffleWriteMetrics( + metrics.shuffleWriteMetrics.bytesWritten, + metrics.shuffleWriteMetrics.writeTime, + metrics.shuffleWriteMetrics.recordsWritten)) + } + + private def newJobDataWrapper( + old: Option[JobDataWrapper], + info: v1.JobData, + initialStages: Option[Seq[JobStageSummary]] = None, + submittedStages: Option[Set[Int]] = None, + completedStages: Option[Set[Int]] = None): JobDataWrapper = { + new JobDataWrapper(info, + value(initialStages, old.map(_.initialStages), Seq()), + value(submittedStages, old.map(_.submittedStages), Set()), + value(completedStages, old.map(_.completedStages), Set())) + } + + private def newStageDataWrapper( + old: Option[StageDataWrapper], + info: v1.StageData, + jobIds: Option[Set[Int]] = None): StageDataWrapper = { + new StageDataWrapper(info, jobIds.orElse(old.map(_.jobIds)).getOrElse(Set())) + } + + private def activeJobs(stage: StageDataWrapper): Set[Int] = { + stage.jobIds.filter(activeJobs.contains(_)) + } + + private def filterAccumulables(accums: Iterable[AccumulableInfo]): Iterable[AccumulableInfo] = { + accums.filter { acc => + // We don't need to store internal or SQL accumulables as their values will be shown in + // other places, so drop them to reduce the memory usage. + !acc.internal && acc.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) + } + } + + private def mergeAccumulables( + old: Seq[v1.AccumulableInfo], + updates: Iterable[AccumulableInfo]): Seq[v1.AccumulableInfo] = { + val newAccums = filterAccumulables(updates) + .map { acc => + acc.id -> newAccumulableInfo(acc) + }.toMap + + val oldAccums = old.map { acc => acc.id -> acc }.toMap + (oldAccums ++ newAccums).values.toSeq.sortBy(_.id) + } + + private def killedTasksSummary( + reason: TaskEndReason, + oldSummary: Map[String, Int]): Map[String, Int] = { + reason match { + case k: TaskKilled => + oldSummary.updated(k.reason, oldSummary.getOrElse(k.reason, 0) + 1) + case _ => + oldSummary + } + } + + private def time(event: AnyRef)(fn: => Unit): Unit = { + val timer = metrics.timer(event.getClass().getName()) + KVUtils.timeIt(timer, anyEventTimer) { + fn + } + } + + private def dumpMetrics(level: Slf4jReporter.LoggingLevel): Unit = { + if (appId != null) { + val log = LoggerFactory.getLogger(getClass().getName() + "." + appId) + Slf4jReporter.forRegistry(metrics) + .outputTo(log) + .withLoggingLevel(level) + .build() + .report() + } + } + +} + +private[spark] object AppStateListener { + + val DEFAULT_DATE = new Date(-1) + val UNKNOWN = "" + +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStateStore.scala b/core/src/main/scala/org/apache/spark/status/AppStateStore.scala new file mode 100644 index 0000000000000..a4da484199cf7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppStateStore.scala @@ -0,0 +1,401 @@ +/* + * 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.status + +import java.io.File +import java.util.{Arrays, List => JList} + +import scala.collection.JavaConverters._ + +import org.apache.spark.{JobExecutionStatus, SparkConf} +import org.apache.spark.kvstore.{KVStore, KVStoreView} +import org.apache.spark.scheduler.{SparkListenerBus, SparkListenerEvent} +import org.apache.spark.status.api.v1 +import org.apache.spark.ui.scope._ +import org.apache.spark.util.{Distribution, Utils} + +/** + * A wrapper around a KVStore that provides methods for accessing the API data stored within. + */ +private[spark] class AppStateStore private ( + store: KVStore, + tempStorePath: Option[File], + listener: Option[AppStateListener]) { + + def appInfo(): v1.ApplicationInfo = { + // There should be a single application info for a UIStore instance, so do no checks here. + val it = store.view(classOf[ApplicationInfoWrapper]).closeableIterator() + try { + it.next().info + } finally { + it.close() + } + } + + def environmentInfo(): v1.ApplicationEnvironmentInfo = { + val klass = classOf[ApplicationEnvironmentInfoWrapper] + store.read(klass, klass.getName()).info + } + + def jobsList(statuses: JList[JobExecutionStatus]): Seq[v1.JobData] = { + val it = store.view(classOf[JobDataWrapper]).reverse().asScala.map(_.info) + if (statuses != null && !statuses.isEmpty()) { + it.filter { job => statuses.contains(job.status) }.toSeq + } else { + it.toSeq + } + } + + def job(jobId: Int): v1.JobData = { + store.read(classOf[JobDataWrapper], jobId).info + } + + def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { + store.view(classOf[ExecutorSummaryWrapper]).index("active").reverse().first(true).last(true) + .asScala.map(_.info).toSeq + } + + def executorSummary(executorId: String): Option[v1.ExecutorSummary] = { + try { + Some(store.read(classOf[ExecutorSummaryWrapper], executorId).info) + } catch { + case _: NoSuchElementException => + None + } + } + + def stageList(statuses: JList[v1.StageStatus]): Seq[v1.StageData] = { + val it = store.view(classOf[StageDataWrapper]).reverse().asScala.map(_.info) + if (statuses != null && !statuses.isEmpty()) { + it.filter { s => statuses.contains(s.status) }.toSeq + } else { + it.toSeq + } + } + + def stageData(stageId: Int): Seq[v1.StageData] = { + store.view(classOf[StageDataWrapper]).index("stageId").first(stageId).last(stageId) + .asScala.map { s => stageWithDetails(s.info) }.toSeq + } + + def lastStageAttempt(stageId: Int): v1.StageData = { + val it = store.view(classOf[StageDataWrapper]).index("stageId").reverse().first(stageId) + .closeableIterator() + try { + it.next().info + } finally { + it.close() + } + } + + def stageAttempt(stageId: Int, stageAttemptId: Int): v1.StageData = { + val stageKey = Array(stageId, stageAttemptId) + val stage = store.read(classOf[StageDataWrapper], stageKey).info + stageWithDetails(stage) + } + + def taskSummary( + stageId: Int, + stageAttemptId: Int, + quantiles: Array[Double]): v1.TaskMetricDistributions = { + + val stage = Array(stageId, stageAttemptId) + + val rawMetrics = store.view(classOf[TaskDataWrapper]) + .index("stage") + .first(stage) + .last(stage) + .asScala + .flatMap(_.info.taskMetrics) + .toList + .view + + def metricQuantiles(f: v1.TaskMetrics => Double): IndexedSeq[Double] = + Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) + + // We need to do a lot of similar munging to nested metrics here. For each one, + // we want (a) extract the values for nested metrics (b) make a distribution for each metric + // (c) shove the distribution into the right field in our return type and (d) only return + // a result if the option is defined for any of the tasks. MetricHelper is a little util + // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just + // implement one "build" method, which just builds the quantiles for each field. + + val inputMetrics = + new MetricHelper[v1.InputMetrics, v1.InputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw: v1.TaskMetrics): v1.InputMetrics = raw.inputMetrics + + def build: v1.InputMetricDistributions = new v1.InputMetricDistributions( + bytesRead = submetricQuantiles(_.bytesRead), + recordsRead = submetricQuantiles(_.recordsRead) + ) + }.build + + val outputMetrics = + new MetricHelper[v1.OutputMetrics, v1.OutputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw: v1.TaskMetrics): v1.OutputMetrics = raw.outputMetrics + + def build: v1.OutputMetricDistributions = new v1.OutputMetricDistributions( + bytesWritten = submetricQuantiles(_.bytesWritten), + recordsWritten = submetricQuantiles(_.recordsWritten) + ) + }.build + + val shuffleReadMetrics = + new MetricHelper[v1.ShuffleReadMetrics, v1.ShuffleReadMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleReadMetrics = + raw.shuffleReadMetrics + + def build: v1.ShuffleReadMetricDistributions = new v1.ShuffleReadMetricDistributions( + readBytes = submetricQuantiles { s => s.localBytesRead + s.remoteBytesRead }, + readRecords = submetricQuantiles(_.recordsRead), + remoteBytesRead = submetricQuantiles(_.remoteBytesRead), + remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), + localBlocksFetched = submetricQuantiles(_.localBlocksFetched), + totalBlocksFetched = submetricQuantiles { s => + s.localBlocksFetched + s.remoteBlocksFetched + }, + fetchWaitTime = submetricQuantiles(_.fetchWaitTime) + ) + }.build + + val shuffleWriteMetrics = + new MetricHelper[v1.ShuffleWriteMetrics, v1.ShuffleWriteMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleWriteMetrics = + raw.shuffleWriteMetrics + + def build: v1.ShuffleWriteMetricDistributions = new v1.ShuffleWriteMetricDistributions( + writeBytes = submetricQuantiles(_.bytesWritten), + writeRecords = submetricQuantiles(_.recordsWritten), + writeTime = submetricQuantiles(_.writeTime) + ) + }.build + + new v1.TaskMetricDistributions( + quantiles = quantiles, + executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), + executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime), + executorRunTime = metricQuantiles(_.executorRunTime), + executorCpuTime = metricQuantiles(_.executorCpuTime), + resultSize = metricQuantiles(_.resultSize), + jvmGcTime = metricQuantiles(_.jvmGcTime), + resultSerializationTime = metricQuantiles(_.resultSerializationTime), + memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled), + diskBytesSpilled = metricQuantiles(_.diskBytesSpilled), + inputMetrics = inputMetrics, + outputMetrics = outputMetrics, + shuffleReadMetrics = shuffleReadMetrics, + shuffleWriteMetrics = shuffleWriteMetrics + ) + } + + def taskList(stageId: Int, stageAttemptId: Int, maxTasks: Int): Seq[v1.TaskData] = { + val stageKey = Array(stageId, stageAttemptId) + store.view(classOf[TaskDataWrapper]).index("stage").first(stageKey).last(stageKey).reverse() + .max(maxTasks).asScala.map(_.info).toSeq.reverse + } + + def taskList( + stageId: Int, + stageAttemptId: Int, + offset: Int, + length: Int, + sortBy: v1.TaskSorting): Seq[v1.TaskData] = { + val stageKey = Array(stageId, stageAttemptId) + val base = store.view(classOf[TaskDataWrapper]) + val indexed = sortBy match { + case v1.TaskSorting.ID => + base.index("stage").first(stageKey).last(stageKey) + case v1.TaskSorting.INCREASING_RUNTIME => + base.index("runtime").first(stageKey ++ Array(-1L)).last(stageKey ++ Array(Long.MaxValue)) + case v1.TaskSorting.DECREASING_RUNTIME => + base.index("runtime").first(stageKey ++ Array(Long.MaxValue)).last(stageKey ++ Array(-1L)) + .reverse() + } + indexed.skip(offset).max(length).asScala.map(_.info).toSeq + } + + private def stageWithDetails(stage: v1.StageData): v1.StageData = { + // TODO: limit tasks returned. + val maxTasks = Int.MaxValue + val tasks = taskList(stage.stageId, stage.attemptId, maxTasks) + .map { t => (t.taskId, t) } + .toMap + + val stageKey = Array(stage.stageId, stage.attemptId) + val execs = store.view(classOf[ExecutorStageSummaryWrapper]) + .index("stage") + .first(stageKey) + .last(stageKey) + .asScala + .map { exec => (exec.executorId -> exec.info) } + .toMap + + new v1.StageData( + stage.status, + stage.stageId, + stage.attemptId, + stage.numTasks, + stage.numActiveTasks, + stage.numCompleteTasks, + stage.numFailedTasks, + stage.numKilledTasks, + stage.executorRunTime, + stage.executorCpuTime, + stage.submissionTime, + stage.firstTaskLaunchedTime, + stage.completionTime, + stage.failureReason, + stage.inputBytes, + stage.inputRecords, + stage.outputBytes, + stage.outputRecords, + stage.shuffleReadBytes, + stage.shuffleReadRecords, + stage.shuffleWriteBytes, + stage.shuffleWriteRecords, + stage.memoryBytesSpilled, + stage.diskBytesSpilled, + stage.name, + stage.description, + stage.details, + stage.schedulingPool, + stage.rddIds, + stage.accumulatorUpdates, + Some(tasks), + Some(execs), + stage.killedTasksSummary) + } + + def rddList(cachedOnly: Boolean = true): Seq[v1.RDDStorageInfo] = { + store.view(classOf[RDDStorageInfoWrapper]).asScala.map(_.info).filter { rdd => + !cachedOnly || rdd.numCachedPartitions > 0 + }.toSeq + } + + def rdd(rddId: Int): v1.RDDStorageInfo = { + store.read(classOf[RDDStorageInfoWrapper], rddId).info + } + + def executorEvents(): Seq[SparkListenerEvent] = { + store.view(classOf[ExecutorEventData]).asScala.map(_.event).toSeq + } + + def streamBlocksList(): Seq[StreamBlockData] = { + store.view(classOf[StreamBlockData]).asScala.toSeq + } + + def operationGraphForStage(stageId: Int): RDDOperationGraph = { + store.read(classOf[RDDOperationGraphWrapper], stageId).toRDDOperationGraph() + } + + def operationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = { + val job = store.read(classOf[JobDataWrapper], jobId) + val stages = job.info.stageIds + val skipped = stages.toSet - job.completedStages + + stages.map { id => + val g = store.read(classOf[RDDOperationGraphWrapper], id).toRDDOperationGraph() + if (skipped.contains(id) && !g.rootCluster.name.contains("skipped")) { + g.rootCluster.setName(g.rootCluster.name + " (skipped)") + } + g + } + } + + def pool(name: String): PoolData = { + store.read(classOf[PoolData], name) + } + + def close(): Unit = { + listener.foreach(_.stop()) + store.close() + tempStorePath.foreach(Utils.deleteRecursively) + } + +} + +private[spark] object AppStateStore { + + import config._ + + val CURRENT_VERSION = 1L + + /** Loads a UI store from the given path, creating an empty store if it doesn't exist. */ + def loadStore(conf: SparkConf, path: File): AppStateStore = { + new AppStateStore(loadStore(path), None, None) + } + + /** + * Crate a state store in a temporary path. A listener will be attached to the given bus to + * populate the store, and the directory will be deleted when the store is closed. + */ + def createTempStore(conf: SparkConf, bus: SparkListenerBus): AppStateStore = { + val temp = Utils.createTempDir(namePrefix = "appstate") + initializeStore(conf, loadStore(temp), Some(temp), bus) + } + + /** + * Create a store in the given path, attaching a listener to the given bus to populate the + * store. The path will not be deleted after the store is closed. + */ + def createStore(path: File, conf: SparkConf, bus: SparkListenerBus): AppStateStore = { + initializeStore(conf, loadStore(path), None, bus) + } + + private def initializeStore( + conf: SparkConf, + store: KVStore, + tempPath: Option[File], + bus: SparkListenerBus): AppStateStore = { + val cachingStore = if (conf.get(MAX_CACHED_ELEMENTS) > 0) { + new AsyncCachingStore(store, conf) + } else { + store + } + val listener = new AppStateListener(cachingStore, conf) + bus.addListener(listener) + new AppStateStore(cachingStore, tempPath, Some(listener)) + } + + private def loadStore(path: File): KVStore = { + val metadata = new AppStatusStoreMetadata(CURRENT_VERSION) + KVUtils.open(path, metadata) + } + +} + +/** + * Helper for getting distributions from nested metric types. + */ +private abstract class MetricHelper[I, O]( + rawMetrics: Seq[v1.TaskMetrics], + quantiles: Array[Double]) { + + def getSubmetrics(raw: v1.TaskMetrics): I + + def build: O + + val data: Seq[I] = rawMetrics.map(getSubmetrics) + + /** applies the given function to all input metrics, and returns the quantiles */ + def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { + Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) + } +} diff --git a/core/src/main/scala/org/apache/spark/status/AsyncCachingStore.scala b/core/src/main/scala/org/apache/spark/status/AsyncCachingStore.scala new file mode 100644 index 0000000000000..395a844cba8a4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AsyncCachingStore.scala @@ -0,0 +1,282 @@ +/* + * 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.status + +import java.util.{Arrays, HashMap, LinkedHashMap, Map => JMap} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.locks.ReentrantLock + +import com.codahale.metrics.{MetricRegistry, Slf4jReporter} +import com.google.common.base.Objects +import com.google.common.cache.CacheBuilder + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.kvstore._ +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * A KVStore implementation that provides an LRU cache of elements to speed up access, and writes + * to the underlying store asynchronously. + * + * Caching is only available for elements read using the [[#read(Class, Object)]] method. Iterators + * are not cached and in fact can return stale data when compared to what the + * [[#read(Class, Object)]] method might return. + * + * Writes are performed on a separate thread, which does de-duplication to avoid redundant writes. + * Because of those features, writers should always update entries based on what the + * [[#read(Class, Object)]] method returns, otherwise they may be operating on stale data. + */ +private class AsyncCachingStore(store: KVStore, conf: SparkConf) extends KVStore with Logging { + + import config._ + import KVUtils._ + + private val cache = CacheBuilder.newBuilder() + .maximumSize(conf.get(MAX_CACHED_ELEMENTS)) + .build[CacheKey, AnyRef]() + + private val writeThread = new Thread(new Runnable() { + override def run(): Unit = Utils.tryLog { + writeThreadImpl() + } + }) + + private val types = new HashMap[Class[_], KVTypeInfo]() + private val writeQueue = new LinkedHashMap[CacheKey, AsyncStoreOp]() + private val writeLock = new ReentrantLock() + private val writeSignal = writeLock.newCondition() + private val maxWriteQueueSize = conf.get(MAX_WRITE_QUEUE_SIZE) + + private val metrics = new MetricRegistry() + private val hits = metrics.counter("hits") + private val misses = metrics.counter("misses") + private val writes = metrics.counter("writes") + private val writeTimer = metrics.timer("writeTimer") + private val maxQueued = new AtomicLong() + + @volatile private var active = true + + writeThread.setName("caching-store-writer") + writeThread.setDaemon(true) + writeThread.start() + + override def read[T](klass: Class[T], naturalKey: AnyRef): T = { + val cacheKey = new CacheKey(klass, naturalKey) + val cached = cache.getIfPresent(cacheKey) + if (cached != null) { + hits.inc() + return cached.asInstanceOf[T] + } + + // There is a possibility that an entry evicted from the cache has pending writes, so check + // the last write op too. + locked(writeLock) { + writeQueue.get(cacheKey) match { + case op: WriteOp => + val value = op.value + if (value != null) { + hits.inc() + return value.asInstanceOf[T] + } + + case _: DeleteOp => + throw new NoSuchElementException() + + case _ => // continue. + } + } + + val stored = store.read(klass, naturalKey) + misses.inc() + cache.put(cacheKey, stored.asInstanceOf[AnyRef]) + stored + } + + override def write(value: AnyRef): Unit = { + var ti = types.get(value.getClass()) + if (ti == null) { + ti = new KVTypeInfo(value.getClass()) + types.put(value.getClass(), ti) + } + + val key = ti.getIndexValue(KVIndex.NATURAL_INDEX_NAME, value) + val cacheKey = new CacheKey(value.getClass(), key) + cache.put(cacheKey, value) + + enqueue(WriteOp(value, cacheKey)) + } + + override def delete(klass: Class[_], naturalKey: AnyRef): Unit = { + val cacheKey = new CacheKey(klass, naturalKey) + cache.invalidate(cacheKey) + enqueue(DeleteOp(cacheKey)) + } + + override def getMetadata[T](klass: Class[T]): T = store.getMetadata(klass) + + override def setMetadata(value: AnyRef): Unit = store.setMetadata(value) + + override def view[T](klass: Class[T]): KVStoreView[T] = store.view(klass) + + override def count(klass: Class[_]): Long = store.count(klass) + + override def close(): Unit = { + try { + val remaining = metrics.counter("writeBacklog") + remaining.inc(writeQueue.size()) + + val timer = metrics.timer("writeQueueDrain") + timeIt(timer) { + active = false + locked(writeLock) { + writeSignal.signal() + } + writeThread.join(TimeUnit.SECONDS.toMillis(30)) + if (writeThread.isAlive()) { + logWarning(s"Write queue hasn't drained in 30 seconds, queue size = ${writeQueue.size()}") + writeThread.interrupt() + } else { + assert(writeQueue.size == 0, "Unwritten items left over in write queue.") + } + } + + metrics.counter("maxQueueSize").inc(maxQueued.get()) + Slf4jReporter.forRegistry(metrics) + .outputTo(log) + .withLoggingLevel(Slf4jReporter.LoggingLevel.DEBUG) + .build() + .report() + + cache.invalidateAll() + } finally { + store.close() + } + } + + private def enqueue(op: AsyncStoreOp): Unit = { + locked(writeLock) { + // If not replacing an existing item in the queue, wait until there's room before queueing + // the write. + if (writeQueue.remove(op.cacheKey) == null) { + while (writeQueue.size() >= maxWriteQueueSize) { + writeSignal.await() + } + } + writeQueue.put(op.cacheKey, op) + maxQueued.set(math.max(writeQueue.size(), maxQueued.get())) + writeSignal.signal() + } + writes.inc() + } + + private def writeThreadImpl(): Unit = { + while (active || !writeQueue.isEmpty()) { + Utils.tryLog { + val next = locked(writeLock) { + while (active && writeQueue.isEmpty()) { + writeSignal.await() + } + + if (!writeQueue.isEmpty()) { + val iter = writeQueue.entrySet().iterator() + val _next = iter.next() + iter.remove() + writeSignal.signal() + _next.getValue() + } else { + null + } + } + + if (next != null) { + timeIt(writeTimer) { + next.perform() + } + } + } + } + } + + // For testing. Halts the write thread. + private[status] def haltWrites(): Unit = { + writeLock.lock() + } + + // For testing. Resumes the write thread. + private[status] def resumeWrites(): Unit = { + writeLock.unlock() + } + + private sealed abstract class AsyncStoreOp(val cacheKey: CacheKey) { + + def perform(): Unit + + } + + private case class WriteOp(value: AnyRef, key: CacheKey) extends AsyncStoreOp(key) { + + override def perform(): Unit = store.write(value) + + } + + private case class DeleteOp(key: CacheKey) extends AsyncStoreOp(key) { + + override def perform(): Unit = store.delete(cacheKey.klass, cacheKey.key) + + } + +} + +private class CacheKey(val klass: Class[_], val key: AnyRef) { + + private val isArray = key != null && key.getClass().isArray() + + override def equals(o: Any): Boolean = { + if (o == null || !o.isInstanceOf[CacheKey]) { + false + } + + val other = o.asInstanceOf[CacheKey] + if (!Objects.equal(klass, other.klass)) { + false + } else if (Objects.equal(key, other.key)) { + true + } else if (isArray && other.key.getClass().isArray()) { + val a1 = key.asInstanceOf[Array[_]] + val a2 = other.key.asInstanceOf[Array[_]] + a1.length == a2.length && + a1.getClass().getComponentType() == a2.getClass().getComponentType() && + a1.zip(a2).forall { case (v1, v2) => v1 == v2 } + } else { + false + } + } + + override def hashCode(): Int = { + if (isArray) { + 31 * key.asInstanceOf[Array[_]].map(_.hashCode()).sum + } else if (key != null) { + key.hashCode() + } else { + 0 + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala new file mode 100644 index 0000000000000..10b71dd8f303b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala @@ -0,0 +1,133 @@ +/* + * 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.status + +import java.io.File +import java.util.concurrent.locks.Lock + +import scala.annotation.meta.getter +import scala.language.implicitConversions +import scala.reflect.{classTag, ClassTag} + +import com.codahale.metrics.Timer +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.kvstore._ +import org.apache.spark.status.api.v1.JacksonMessageWriter + +/** A mix-in trait with helper methods to read and update data from a KVStore. */ +private[spark] trait KVUtils { + + protected val kvstore: KVStore + + /** Helper method for choosing among a new, old or default value. */ + protected def value[T](newv: Option[T], oldv: Option[T], dflt: T): T = { + newv.orElse(oldv).getOrElse(dflt) + } + + /** Helper method for choosing between an optional new or old value. */ + protected def option[T](newv: Option[Option[T]], oldv: Option[Option[T]]): Option[T] = { + newv.getOrElse(oldv.getOrElse(None)) + } + + /** Helper method for reading a value from a KVStore, and return an Option. */ + protected def read[T: ClassTag](key: Any): Option[T] = { + try { + Some(kvstore.read(classTag[T].runtimeClass, key).asInstanceOf[T]) + } catch { + case _: NoSuchElementException => None + } + } + + /** Helper method for updating a value read from a KVStore. */ + protected def update[T: ClassTag](key: Any)(fn: Option[T] => T): Unit = { + val updated = fn(read(key)) + kvstore.write(updated) + } + + /** Utility conversion method used to copy immutable classes. */ + protected implicit def anyToOption[T](o: T): Option[T] = Option(o) + +} + +private[spark] object KVUtils extends Logging { + + /** Use this to annotate constructor params to be used as KVStore indices. */ + type KVIndexParam = KVIndex @getter + + /** + * A KVStoreSerializer that provides Scala types serialization too, and uses the same options as + * the API serializer. + */ + class KVStoreScalaSerializer extends KVStoreSerializer { + + mapper.registerModule(DefaultScalaModule) + mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) + mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) + + } + + /** + * Open or create a LevelDB store. + * + * @param path Location of the store. + * @param metadata Metadata value to compare to the data in the store. If the store does not + * contain any metadata (e.g. it's a new store), this value is written as + * the store's metadata. + */ + def open[M: ClassTag](path: File, metadata: M): LevelDB = { + require(metadata != null, "Metadata is required.") + + val db = new LevelDB(path, new KVStoreScalaSerializer()) + val dbMeta = db.getMetadata(classTag[M].runtimeClass) + if (dbMeta == null) { + db.setMetadata(metadata) + } else if (dbMeta != metadata) { + db.close() + throw new MetadataMismatchException() + } + + db + } + + /** Measure the run time of a closure, updating the given timers. */ + def timeIt[T](timers: Timer*)(fn: => T): T = { + val ctx = timers.map(_.time()).toList + try { + fn + } finally { + ctx.foreach(_.stop()) + } + } + + /** Runs a closure while holding a lock. */ + def locked[T](lock: Lock)(fn: => T): T = { + lock.lock() + try { + fn + } finally { + lock.unlock() + } + } + + class MetadataMismatchException extends Exception + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala deleted file mode 100644 index 01f2a18122e6f..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala +++ /dev/null @@ -1,41 +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.status.api.v1 - -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.exec.ExecutorsPage - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllExecutorListResource(ui: SparkUI) { - - @GET - def executorList(): Seq[ExecutorSummary] = { - val listener = ui.executorsListener - listener.synchronized { - // The follow codes should be protected by `listener` to make sure no executors will be - // removed before we query their status. See SPARK-12784. - (0 until listener.activeStorageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId, isActive = true) - } ++ (0 until listener.deadStorageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId, isActive = false) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala deleted file mode 100644 index d0d9ef1165e81..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala +++ /dev/null @@ -1,103 +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.status.api.v1 - -import java.util.{Arrays, Date, List => JList} -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -import org.apache.spark.JobExecutionStatus -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.jobs.UIData.JobUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllJobsResource(ui: SparkUI) { - - @GET - def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = { - val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = - AllJobsResource.getStatusToJobs(ui) - val adjStatuses: JList[JobExecutionStatus] = { - if (statuses.isEmpty) { - Arrays.asList(JobExecutionStatus.values(): _*) - } else { - statuses - } - } - val jobInfos = for { - (status, jobs) <- statusToJobs - job <- jobs if adjStatuses.contains(status) - } yield { - AllJobsResource.convertJobData(job, ui.jobProgressListener, false) - } - jobInfos.sortBy{- _.jobId} - } - -} - -private[v1] object AllJobsResource { - - def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = { - val statusToJobs = ui.jobProgressListener.synchronized { - Seq( - JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq, - JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq, - JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq - ) - } - statusToJobs - } - - def convertJobData( - job: JobUIData, - listener: JobProgressListener, - includeStageDetails: Boolean): JobData = { - listener.synchronized { - val lastStageInfo = - if (job.stageIds.isEmpty) { - None - } else { - listener.stageIdToInfo.get(job.stageIds.max) - } - val lastStageData = lastStageInfo.flatMap { s => - listener.stageIdToData.get((s.stageId, s.attemptId)) - } - val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap { _.description } - new JobData( - jobId = job.jobId, - name = lastStageName, - description = lastStageDescription, - submissionTime = job.submissionTime.map{new Date(_)}, - completionTime = job.completionTime.map{new Date(_)}, - stageIds = job.stageIds, - jobGroup = job.jobGroup, - status = job.status, - numTasks = job.numTasks, - numActiveTasks = job.numActiveTasks, - numCompletedTasks = job.numCompletedTasks, - numSkippedTasks = job.numSkippedTasks, - numFailedTasks = job.numFailedTasks, - numActiveStages = job.numActiveStages, - numCompletedStages = job.completedStageIndices.size, - numSkippedStages = job.numSkippedStages, - numFailedStages = job.numFailedStages - ) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala deleted file mode 100644 index 1279b281ad8d8..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ /dev/null @@ -1,110 +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.status.api.v1 - -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils} -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.storage.StorageListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllRDDResource(ui: SparkUI) { - - @GET - def rddList(): Seq[RDDStorageInfo] = { - val storageStatusList = ui.storageListener.activeStorageStatusList - val rddInfos = ui.storageListener.rddInfoList - rddInfos.map{rddInfo => - AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList, - includeDetails = false) - } - } - -} - -private[spark] object AllRDDResource { - - def getRDDStorageInfo( - rddId: Int, - listener: StorageListener, - includeDetails: Boolean): Option[RDDStorageInfo] = { - val storageStatusList = listener.activeStorageStatusList - listener.rddInfoList.find { _.id == rddId }.map { rddInfo => - getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails) - } - } - - def getRDDStorageInfo( - rddId: Int, - rddInfo: RDDInfo, - storageStatusList: Seq[StorageStatus], - includeDetails: Boolean): RDDStorageInfo = { - val workers = storageStatusList.map { (rddId, _) } - val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) - val blocks = storageStatusList - .flatMap { _.rddBlocksById(rddId) } - .sortWith { _._1.name < _._1.name } - .map { case (blockId, status) => - (blockId, status, blockLocations.getOrElse(blockId, Seq[String]("Unknown"))) - } - - val dataDistribution = if (includeDetails) { - Some(storageStatusList.map { status => - new RDDDataDistribution( - address = status.blockManagerId.hostPort, - memoryUsed = status.memUsedByRdd(rddId), - memoryRemaining = status.memRemaining, - diskUsed = status.diskUsedByRdd(rddId), - onHeapMemoryUsed = Some( - if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), - offHeapMemoryUsed = Some( - if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), - onHeapMemoryRemaining = status.onHeapMemRemaining, - offHeapMemoryRemaining = status.offHeapMemRemaining - ) } ) - } else { - None - } - val partitions = if (includeDetails) { - Some(blocks.map { case (id, block, locations) => - new RDDPartitionInfo( - blockName = id.name, - storageLevel = block.storageLevel.description, - memoryUsed = block.memSize, - diskUsed = block.diskSize, - executors = locations - ) - } ) - } else { - None - } - - new RDDStorageInfo( - id = rddId, - name = rddInfo.name, - numPartitions = rddInfo.numPartitions, - numCachedPartitions = rddInfo.numCachedPartitions, - storageLevel = rddInfo.storageLevel.description, - memoryUsed = rddInfo.memSize, - diskUsed = rddInfo.diskSize, - dataDistribution = dataDistribution, - partitions = partitions - ) - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala deleted file mode 100644 index 1818935392eb3..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ /dev/null @@ -1,315 +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.status.api.v1 - -import java.util.{Arrays, Date, List => JList} -import javax.ws.rs.{GET, Produces, QueryParam} -import javax.ws.rs.core.MediaType - -import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} -import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} -import org.apache.spark.util.Distribution - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllStagesResource(ui: SparkUI) { - - @GET - def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { - val listener = ui.jobProgressListener - val stageAndStatus = AllStagesResource.stagesAndStatus(ui) - val adjStatuses = { - if (statuses.isEmpty()) { - Arrays.asList(StageStatus.values(): _*) - } else { - statuses - } - } - for { - (status, stageList) <- stageAndStatus - stageInfo: StageInfo <- stageList if adjStatuses.contains(status) - stageUiData: StageUIData <- listener.synchronized { - listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId)) - } - } yield { - AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) - } - } -} - -private[v1] object AllStagesResource { - def stageUiToStageData( - status: StageStatus, - stageInfo: StageInfo, - stageUiData: StageUIData, - includeDetails: Boolean): StageData = { - - val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) - - val firstTaskLaunchedTime: Option[Date] = - if (taskLaunchTimes.nonEmpty) { - Some(new Date(taskLaunchTimes.min)) - } else { - None - } - - val taskData = if (includeDetails) { - Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } ) - } else { - None - } - val executorSummary = if (includeDetails) { - Some(stageUiData.executorSummary.map { case (k, summary) => - k -> new ExecutorStageSummary( - taskTime = summary.taskTime, - failedTasks = summary.failedTasks, - succeededTasks = summary.succeededTasks, - inputBytes = summary.inputBytes, - outputBytes = summary.outputBytes, - shuffleRead = summary.shuffleRead, - shuffleWrite = summary.shuffleWrite, - memoryBytesSpilled = summary.memoryBytesSpilled, - diskBytesSpilled = summary.diskBytesSpilled - ) - }) - } else { - None - } - - val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq - - new StageData( - status = status, - stageId = stageInfo.stageId, - attemptId = stageInfo.attemptId, - numActiveTasks = stageUiData.numActiveTasks, - numCompleteTasks = stageUiData.numCompleteTasks, - numFailedTasks = stageUiData.numFailedTasks, - executorRunTime = stageUiData.executorRunTime, - executorCpuTime = stageUiData.executorCpuTime, - submissionTime = stageInfo.submissionTime.map(new Date(_)), - firstTaskLaunchedTime, - completionTime = stageInfo.completionTime.map(new Date(_)), - inputBytes = stageUiData.inputBytes, - inputRecords = stageUiData.inputRecords, - outputBytes = stageUiData.outputBytes, - outputRecords = stageUiData.outputRecords, - shuffleReadBytes = stageUiData.shuffleReadTotalBytes, - shuffleReadRecords = stageUiData.shuffleReadRecords, - shuffleWriteBytes = stageUiData.shuffleWriteBytes, - shuffleWriteRecords = stageUiData.shuffleWriteRecords, - memoryBytesSpilled = stageUiData.memoryBytesSpilled, - diskBytesSpilled = stageUiData.diskBytesSpilled, - schedulingPool = stageUiData.schedulingPool, - name = stageInfo.name, - details = stageInfo.details, - accumulatorUpdates = accumulableInfo, - tasks = taskData, - executorSummary = executorSummary - ) - } - - def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = { - val listener = ui.jobProgressListener - listener.synchronized { - Seq( - StageStatus.ACTIVE -> listener.activeStages.values.toSeq, - StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq, - StageStatus.FAILED -> listener.failedStages.reverse.toSeq, - StageStatus.PENDING -> listener.pendingStages.values.toSeq - ) - } - } - - def convertTaskData(uiData: TaskUIData): TaskData = { - new TaskData( - taskId = uiData.taskInfo.taskId, - index = uiData.taskInfo.index, - attempt = uiData.taskInfo.attemptNumber, - launchTime = new Date(uiData.taskInfo.launchTime), - duration = uiData.taskDuration, - executorId = uiData.taskInfo.executorId, - host = uiData.taskInfo.host, - status = uiData.taskInfo.status, - taskLocality = uiData.taskInfo.taskLocality.toString(), - speculative = uiData.taskInfo.speculative, - accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, - errorMessage = uiData.errorMessage, - taskMetrics = uiData.metrics.map { convertUiTaskMetrics } - ) - } - - def taskMetricDistributions( - allTaskData: Iterable[TaskUIData], - quantiles: Array[Double]): TaskMetricDistributions = { - - val rawMetrics = allTaskData.flatMap{_.metrics}.toSeq - - def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = - Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) - - // We need to do a lot of similar munging to nested metrics here. For each one, - // we want (a) extract the values for nested metrics (b) make a distribution for each metric - // (c) shove the distribution into the right field in our return type and (d) only return - // a result if the option is defined for any of the tasks. MetricHelper is a little util - // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just - // implement one "build" method, which just builds the quantiles for each field. - - val inputMetrics: InputMetricDistributions = - new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalInputMetrics = raw.inputMetrics - - def build: InputMetricDistributions = new InputMetricDistributions( - bytesRead = submetricQuantiles(_.bytesRead), - recordsRead = submetricQuantiles(_.recordsRead) - ) - }.build - - val outputMetrics: OutputMetricDistributions = - new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalOutputMetrics = raw.outputMetrics - - def build: OutputMetricDistributions = new OutputMetricDistributions( - bytesWritten = submetricQuantiles(_.bytesWritten), - recordsWritten = submetricQuantiles(_.recordsWritten) - ) - }.build - - val shuffleReadMetrics: ShuffleReadMetricDistributions = - new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleReadMetrics = - raw.shuffleReadMetrics - - def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( - readBytes = submetricQuantiles(_.totalBytesRead), - readRecords = submetricQuantiles(_.recordsRead), - remoteBytesRead = submetricQuantiles(_.remoteBytesRead), - remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), - localBlocksFetched = submetricQuantiles(_.localBlocksFetched), - totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), - fetchWaitTime = submetricQuantiles(_.fetchWaitTime) - ) - }.build - - val shuffleWriteMetrics: ShuffleWriteMetricDistributions = - new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleWriteMetrics = - raw.shuffleWriteMetrics - - def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.bytesWritten), - writeRecords = submetricQuantiles(_.recordsWritten), - writeTime = submetricQuantiles(_.writeTime) - ) - }.build - - new TaskMetricDistributions( - quantiles = quantiles, - executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), - executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime), - executorRunTime = metricQuantiles(_.executorRunTime), - executorCpuTime = metricQuantiles(_.executorCpuTime), - resultSize = metricQuantiles(_.resultSize), - jvmGcTime = metricQuantiles(_.jvmGCTime), - resultSerializationTime = metricQuantiles(_.resultSerializationTime), - memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled), - diskBytesSpilled = metricQuantiles(_.diskBytesSpilled), - inputMetrics = inputMetrics, - outputMetrics = outputMetrics, - shuffleReadMetrics = shuffleReadMetrics, - shuffleWriteMetrics = shuffleWriteMetrics - ) - } - - def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { - new AccumulableInfo( - acc.id, acc.name.orNull, acc.update.map(_.toString), acc.value.map(_.toString).orNull) - } - - def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { - new TaskMetrics( - executorDeserializeTime = internal.executorDeserializeTime, - executorDeserializeCpuTime = internal.executorDeserializeCpuTime, - executorRunTime = internal.executorRunTime, - executorCpuTime = internal.executorCpuTime, - resultSize = internal.resultSize, - jvmGcTime = internal.jvmGCTime, - resultSerializationTime = internal.resultSerializationTime, - memoryBytesSpilled = internal.memoryBytesSpilled, - diskBytesSpilled = internal.diskBytesSpilled, - inputMetrics = convertInputMetrics(internal.inputMetrics), - outputMetrics = convertOutputMetrics(internal.outputMetrics), - shuffleReadMetrics = convertShuffleReadMetrics(internal.shuffleReadMetrics), - shuffleWriteMetrics = convertShuffleWriteMetrics(internal.shuffleWriteMetrics) - ) - } - - def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = { - new InputMetrics( - bytesRead = internal.bytesRead, - recordsRead = internal.recordsRead - ) - } - - def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = { - new OutputMetrics( - bytesWritten = internal.bytesWritten, - recordsWritten = internal.recordsWritten - ) - } - - def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = { - new ShuffleReadMetrics( - remoteBlocksFetched = internal.remoteBlocksFetched, - localBlocksFetched = internal.localBlocksFetched, - fetchWaitTime = internal.fetchWaitTime, - remoteBytesRead = internal.remoteBytesRead, - localBytesRead = internal.localBytesRead, - recordsRead = internal.recordsRead - ) - } - - def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { - new ShuffleWriteMetrics( - bytesWritten = internal.bytesWritten, - writeTime = internal.writeTime, - recordsWritten = internal.recordsWritten - ) - } -} - -/** - * Helper for getting distributions from nested metric types. - */ -private[v1] abstract class MetricHelper[I, O]( - rawMetrics: Seq[InternalTaskMetrics], - quantiles: Array[Double]) { - - def getSubmetrics(raw: InternalTaskMetrics): I - - def build: O - - val data: Seq[I] = rawMetrics.map(getSubmetrics) - - /** applies the given function to all input metrics, and returns the quantiles */ - def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { - Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index f17b637754826..ed9bdc6e1e3c2 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -44,189 +44,14 @@ import org.apache.spark.ui.SparkUI private[v1] class ApiRootResource extends ApiRequestContext { @Path("applications") - def getApplicationList(): ApplicationListResource = { - new ApplicationListResource(uiRoot) - } + def applicationList(): Class[ApplicationListResource] = classOf[ApplicationListResource] @Path("applications/{appId}") - def getApplication(): OneApplicationResource = { - new OneApplicationResource(uiRoot) - } - - @Path("applications/{appId}/{attemptId}/jobs") - def getJobs( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllJobsResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new AllJobsResource(ui) - } - } - - @Path("applications/{appId}/jobs") - def getJobs(@PathParam("appId") appId: String): AllJobsResource = { - withSparkUI(appId, None) { ui => - new AllJobsResource(ui) - } - } - - @Path("applications/{appId}/jobs/{jobId: \\d+}") - def getJob(@PathParam("appId") appId: String): OneJobResource = { - withSparkUI(appId, None) { ui => - new OneJobResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}") - def getJob( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneJobResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new OneJobResource(ui) - } - } - - @Path("applications/{appId}/executors") - def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = { - withSparkUI(appId, None) { ui => - new ExecutorListResource(ui) - } - } - - @Path("applications/{appId}/allexecutors") - def getAllExecutors(@PathParam("appId") appId: String): AllExecutorListResource = { - withSparkUI(appId, None) { ui => - new AllExecutorListResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/executors") - def getExecutors( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): ExecutorListResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new ExecutorListResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/allexecutors") - def getAllExecutors( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllExecutorListResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new AllExecutorListResource(ui) - } - } - - @Path("applications/{appId}/stages") - def getStages(@PathParam("appId") appId: String): AllStagesResource = { - withSparkUI(appId, None) { ui => - new AllStagesResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/stages") - def getStages( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllStagesResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new AllStagesResource(ui) - } - } - - @Path("applications/{appId}/stages/{stageId: \\d+}") - def getStage(@PathParam("appId") appId: String): OneStageResource = { - withSparkUI(appId, None) { ui => - new OneStageResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}") - def getStage( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneStageResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new OneStageResource(ui) - } - } - - @Path("applications/{appId}/storage/rdd") - def getRdds(@PathParam("appId") appId: String): AllRDDResource = { - withSparkUI(appId, None) { ui => - new AllRDDResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/storage/rdd") - def getRdds( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllRDDResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new AllRDDResource(ui) - } - } - - @Path("applications/{appId}/storage/rdd/{rddId: \\d+}") - def getRdd(@PathParam("appId") appId: String): OneRDDResource = { - withSparkUI(appId, None) { ui => - new OneRDDResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}") - def getRdd( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneRDDResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new OneRDDResource(ui) - } - } - - @Path("applications/{appId}/logs") - def getEventLogs( - @PathParam("appId") appId: String): EventLogDownloadResource = { - try { - // withSparkUI will throw NotFoundException if attemptId exists for this application. - // So we need to try again with attempt id "1". - withSparkUI(appId, None) { _ => - new EventLogDownloadResource(uiRoot, appId, None) - } - } catch { - case _: NotFoundException => - withSparkUI(appId, Some("1")) { _ => - new EventLogDownloadResource(uiRoot, appId, None) - } - } - } - - @Path("applications/{appId}/{attemptId}/logs") - def getEventLogs( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { - withSparkUI(appId, Some(attemptId)) { _ => - new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) - } - } + def application(): Class[OneApplicationResource] = classOf[OneApplicationResource] @Path("version") - def getVersion(): VersionResource = { - new VersionResource(uiRoot) - } + def version(): VersionInfo = new VersionInfo(org.apache.spark.SPARK_VERSION) - @Path("applications/{appId}/environment") - def getEnvironment(@PathParam("appId") appId: String): ApplicationEnvironmentResource = { - withSparkUI(appId, None) { ui => - new ApplicationEnvironmentResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/environment") - def getEnvironment( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): ApplicationEnvironmentResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new ApplicationEnvironmentResource(ui) - } - } } private[spark] object ApiRootResource { @@ -248,7 +73,13 @@ private[spark] object ApiRootResource { * interface needed for them all to expose application info as json. */ private[spark] trait UIRoot { - def getSparkUI(appKey: String): Option[SparkUI] + /** + * Runs some code with the current SparkUI instance for the app / attempt. + * + * @throws NoSuchElementException If the app / attempt pair does not exist. + */ + def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T + def getApplicationInfoList: Iterator[ApplicationInfo] def getApplicationInfo(appId: String): Option[ApplicationInfo] @@ -287,21 +118,30 @@ private[v1] trait ApiRequestContext { def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext) +} - /** - * Get the spark UI with the given appID, and apply a function - * to it. If there is no such app, throw an appropriate exception - */ - def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { - val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) - uiRoot.getSparkUI(appKey) match { - case Some(ui) => +/** + * Base class for resource handlers that use app-specific data. Abstracts away dealing with + * application and attempt IDs, and finding the app's UI. + */ +private[v1] trait BaseAppResource extends ApiRequestContext { + + @PathParam("appId") protected[this] var appId: String = _ + @PathParam("attemptId") protected[this] var attemptId: String = _ + + protected def withUI[T](fn: SparkUI => T): T = { + try { + uiRoot.withSparkUI(appId, Option(attemptId)) { ui => val user = httpRequest.getRemoteUser() if (!ui.securityManager.checkUIViewPermissions(user)) { throw new ForbiddenException(raw"""user "$user" is not authorized""") } - f(ui) - case None => throw new NotFoundException("no such app: " + appId) + fn(ui) + } + } catch { + case _: NoSuchElementException => + val appKey = Option(attemptId).map(appId + "/" + _).getOrElse(appId) + throw new NotFoundException(s"no such app: $appKey") } } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala deleted file mode 100644 index 739a8aceae861..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala +++ /dev/null @@ -1,45 +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.status.api.v1 - -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -import org.apache.spark.ui.SparkUI - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class ApplicationEnvironmentResource(ui: SparkUI) { - - @GET - def getEnvironmentInfo(): ApplicationEnvironmentInfo = { - val listener = ui.environmentListener - listener.synchronized { - val jvmInfo = Map(listener.jvmInformation: _*) - val runtime = new RuntimeInfo( - jvmInfo("Java Version"), - jvmInfo("Java Home"), - jvmInfo("Scala Version")) - - new ApplicationEnvironmentInfo( - runtime, - listener.sparkProperties, - listener.systemProperties, - listener.classpathEntries) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index a0239266d8756..27c780ce81428 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -23,7 +23,7 @@ import javax.ws.rs.core.MediaType import org.apache.spark.deploy.history.ApplicationHistoryInfo @Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class ApplicationListResource(uiRoot: UIRoot) { +private[v1] class ApplicationListResource extends ApiRequestContext { @GET def appList( diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala deleted file mode 100644 index c84022ddfeef0..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ /dev/null @@ -1,71 +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.status.api.v1 - -import java.io.OutputStream -import java.util.zip.ZipOutputStream -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.{MediaType, Response, StreamingOutput} - -import scala.util.control.NonFatal - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.Logging - -@Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) -private[v1] class EventLogDownloadResource( - val uIRoot: UIRoot, - val appId: String, - val attemptId: Option[String]) extends Logging { - val conf = SparkHadoopUtil.get.newConfiguration(new SparkConf) - - @GET - def getEventLogs(): Response = { - try { - val fileName = { - attemptId match { - case Some(id) => s"eventLogs-$appId-$id.zip" - case None => s"eventLogs-$appId.zip" - } - } - - val stream = new StreamingOutput { - override def write(output: OutputStream): Unit = { - val zipStream = new ZipOutputStream(output) - try { - uIRoot.writeEventLogs(appId, attemptId, zipStream) - } finally { - zipStream.close() - } - - } - } - - Response.ok(stream) - .header("Content-Disposition", s"attachment; filename=$fileName") - .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) - .build() - } catch { - case NonFatal(e) => - Response.serverError() - .entity(s"Event logs are not available for app: $appId.") - .status(Response.Status.SERVICE_UNAVAILABLE) - .build() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala deleted file mode 100644 index ab53881594180..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ /dev/null @@ -1,40 +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.status.api.v1 - -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.exec.ExecutorsPage - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class ExecutorListResource(ui: SparkUI) { - - @GET - def executorList(): Seq[ExecutorSummary] = { - val listener = ui.executorsListener - listener.synchronized { - // The follow codes should be protected by `listener` to make sure no executors will be - // removed before we query their status. See SPARK-12784. - val storageStatusList = listener.activeStorageStatusList - (0 until storageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId, isActive = true) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index 18c3e2f407360..bd4df07e7afc6 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -16,16 +16,150 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType +import java.io.OutputStream +import java.util.{List => JList} +import java.util.zip.ZipOutputStream +import javax.ws.rs.{GET, Path, PathParam, Produces, QueryParam} +import javax.ws.rs.core.{MediaType, Response, StreamingOutput} + +import scala.util.control.NonFatal + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.SparkUI @Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneApplicationResource(uiRoot: UIRoot) { +private[v1] class AbstractApplicationResource extends BaseAppResource { + + @GET + @Path("jobs") + def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = { + withUI(_.store.jobsList(statuses)) + } + + @GET + @Path("jobs/{jobId: \\d+}") + def oneJob(@PathParam("jobId") jobId: Int): JobData = withUI { ui => + try { + ui.store.job(jobId) + } catch { + case _: NoSuchElementException => + throw new NotFoundException("unknown job: " + jobId) + } + } + + @GET + @Path("executors") + def executorList(): Seq[ExecutorSummary] = withUI(_.store.executorList(true)) + + @GET + @Path("allexecutors") + def allExecutorList(): Seq[ExecutorSummary] = withUI(_.store.executorList(false)) + + @Path("stages") + def stages(): Class[StagesResource] = classOf[StagesResource] + + @GET + @Path("storage/rdd") + def rddList(): Seq[RDDStorageInfo] = withUI(_.store.rddList()) + + @GET + @Path("storage/rdd/{rddId: \\d+}") + def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = withUI { ui => + try { + ui.store.rdd(rddId) + } catch { + case _: NoSuchElementException => + throw new NotFoundException(s"no rdd found w/ id $rddId") + } + } + + @GET + @Path("environment") + def environmentInfo(): ApplicationEnvironmentInfo = withUI(_.store.environmentInfo()) + + @GET + @Path("logs") + @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + def getEventLogs(): Response = { + // Retrieve the UI for the application just to do access permission checks. For backwards + // compatibility, this code also tries with attemptId "1" if the UI without an attempt ID does + // not exist. + try { + withUI { _ => } + } catch { + case _: NotFoundException if attemptId == null => + attemptId = "1" + withUI { _ => } + attemptId = null + } + + try { + val fileName = if (attemptId != null) { + s"eventLogs-$appId-$attemptId.zip" + } else { + s"eventLogs-$appId.zip" + } + + val stream = new StreamingOutput { + override def write(output: OutputStream): Unit = { + val zipStream = new ZipOutputStream(output) + try { + uiRoot.writeEventLogs(appId, Option(attemptId), zipStream) + } finally { + zipStream.close() + } + + } + } + + Response.ok(stream) + .header("Content-Disposition", s"attachment; filename=$fileName") + .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) + .build() + } catch { + case NonFatal(e) => + Response.serverError() + .entity(s"Event logs are not available for app: $appId.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() + } + } + + /** + * This method needs to be last, otherwise it clashes with the paths for the above methods + * and causes JAX-RS to not find things. + */ + @Path("{attemptId}") + def applicationAttempt(): Class[OneApplicationAttemptResource] = { + if (attemptId != null) { + throw new NotFoundException(httpRequest.getRequestURI()) + } + classOf[OneApplicationAttemptResource] + } + +} + +private[v1] class OneApplicationResource extends AbstractApplicationResource { + + @GET + def getApp(): ApplicationInfo = { + val app = uiRoot.getApplicationInfo(appId) + app.getOrElse(throw new NotFoundException("unknown app: " + appId)) + } + +} + +private[v1] class OneApplicationAttemptResource extends AbstractApplicationResource { @GET - def getApp(@PathParam("appId") appId: String): ApplicationInfo = { - val apps = uiRoot.getApplicationInfo(appId) - apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) + def getAttempt(): ApplicationAttemptInfo = { + uiRoot.getApplicationInfo(appId) + .flatMap { app => + app.attempts.filter(_.attemptId == attemptId).headOption + } + .getOrElse { + throw new NotFoundException(s"unknown app $appId, attempt $attemptId") + } } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala deleted file mode 100644 index 653150385c732..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala +++ /dev/null @@ -1,41 +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.status.api.v1 - -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.JobExecutionStatus -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.JobUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneJobResource(ui: SparkUI) { - - @GET - def oneJob(@PathParam("jobId") jobId: Int): JobData = { - val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = - AllJobsResource.getStatusToJobs(ui) - val jobOpt = statusToJobs.flatMap(_._2).find { jobInfo => jobInfo.jobId == jobId} - jobOpt.map { job => - AllJobsResource.convertJobData(job, ui.jobProgressListener, false) - }.getOrElse { - throw new NotFoundException("unknown job: " + jobId) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala deleted file mode 100644 index 3e6d2942d0fbb..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala +++ /dev/null @@ -1,150 +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.status.api.v1 - -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -import org.apache.spark.SparkException -import org.apache.spark.scheduler.StageInfo -import org.apache.spark.status.api.v1.StageStatus._ -import org.apache.spark.status.api.v1.TaskSorting._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.jobs.UIData.StageUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneStageResource(ui: SparkUI) { - - @GET - @Path("") - def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { - withStage(stageId) { stageAttempts => - stageAttempts.map { stage => - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) - } - } - } - - @GET - @Path("/{stageAttemptId: \\d+}") - def oneAttemptData( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { - withStageAttempt(stageId, stageAttemptId) { stage => - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) - } - } - - @GET - @Path("/{stageAttemptId: \\d+}/taskSummary") - def taskSummary( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int, - @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) - : TaskMetricDistributions = { - withStageAttempt(stageId, stageAttemptId) { stage => - val quantiles = quantileString.split(",").map { s => - try { - s.toDouble - } catch { - case nfe: NumberFormatException => - throw new BadParameterException("quantiles", "double", s) - } - } - AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles) - } - } - - @GET - @Path("/{stageAttemptId: \\d+}/taskList") - def taskList( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int, - @DefaultValue("0") @QueryParam("offset") offset: Int, - @DefaultValue("20") @QueryParam("length") length: Int, - @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { - withStageAttempt(stageId, stageAttemptId) { stage => - val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq - .sorted(OneStageResource.ordering(sortBy)) - tasks.slice(offset, offset + length) - } - } - - private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData) - - private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = { - val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId) - if (stageAttempts.isEmpty) { - throw new NotFoundException("unknown stage: " + stageId) - } else { - f(stageAttempts) - } - } - - private def findStageStatusUIData( - listener: JobProgressListener, - stageId: Int): Seq[StageStatusInfoUi] = { - listener.synchronized { - def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = { - infos.filter { _.stageId == stageId }.map { info => - val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId), - // this is an internal error -- we should always have uiData - throw new SparkException( - s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}") - ) - StageStatusInfoUi(status, info, ui) - } - } - getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++ - getStatusInfoUi(COMPLETE, listener.completedStages) ++ - getStatusInfoUi(FAILED, listener.failedStages) ++ - getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq) - } - } - - private def withStageAttempt[T]( - stageId: Int, - stageAttemptId: Int) - (f: StageStatusInfoUi => T): T = { - withStage(stageId) { attempts => - val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId } - oneAttempt match { - case Some(stage) => - f(stage) - case None => - val stageAttempts = attempts.map { _.info.attemptId } - throw new NotFoundException(s"unknown attempt for stage $stageId. " + - s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}") - } - } - } -} - -object OneStageResource { - def ordering(taskSorting: TaskSorting): Ordering[TaskData] = { - val extractor: (TaskData => Long) = td => - taskSorting match { - case ID => td.taskId - case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) - case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) - } - Ordering.by(extractor) - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala new file mode 100644 index 0000000000000..64968c321f0a0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -0,0 +1,91 @@ +/* + * 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.status.api.v1 + +import java.util.{List => JList} +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.SparkException +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.api.v1.StageStatus._ +import org.apache.spark.status.api.v1.TaskSorting._ +import org.apache.spark.ui.SparkUI + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class StagesResource extends BaseAppResource { + + @GET + def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { + withUI(_.store.stageList(statuses)) + } + + @GET + @Path("{stageId: \\d+}") + def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = withUI { ui => + val ret = ui.store.stageData(stageId) + if (ret.nonEmpty) { + ret + } else { + throw new NotFoundException(s"unknown stage: $stageId") + } + } + + @GET + @Path("{stageId: \\d+}/{stageAttemptId: \\d+}") + def oneAttemptData( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int): StageData = withUI { ui => + try { + ui.store.stageAttempt(stageId, stageAttemptId) + } catch { + case _: NoSuchElementException => + throw new NotFoundException(s"unknown attempt $stageAttemptId for stage $stageId.") + } + } + + @GET + @Path("{stageId: \\d+}/{stageAttemptId: \\d+}/taskSummary") + def taskSummary( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) + : TaskMetricDistributions = withUI { ui => + val quantiles = quantileString.split(",").map { s => + try { + s.toDouble + } catch { + case nfe: NumberFormatException => + throw new BadParameterException("quantiles", "double", s) + } + } + + ui.store.taskSummary(stageId, stageAttemptId, quantiles) + } + + @GET + @Path("{stageId: \\d+}/{stageAttemptId: \\d+}/taskList") + def taskList( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0") @QueryParam("offset") offset: Int, + @DefaultValue("20") @QueryParam("length") length: Int, + @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { + withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala deleted file mode 100644 index 673da1ce36b57..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala +++ /dev/null @@ -1,30 +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.status.api.v1 - -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class VersionResource(ui: UIRoot) { - - @GET - def getVersionInfo(): VersionInfo = new VersionInfo( - org.apache.spark.SPARK_VERSION - ) - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 56d8e51732ffd..09d220085e8c3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -16,9 +16,10 @@ */ package org.apache.spark.status.api.v1 +import java.lang.{Long => JLong} import java.util.Date -import scala.collection.Map +import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus @@ -39,19 +40,36 @@ class ApplicationAttemptInfo private[spark]( val duration: Long, val sparkUser: String, val completed: Boolean = false) { - def getStartTimeEpoch: Long = startTime.getTime - def getEndTimeEpoch: Long = endTime.getTime - def getLastUpdatedEpoch: Long = lastUpdated.getTime + + def getStartTimeEpoch: Long = startTime.getTime + + def getEndTimeEpoch: Long = endTime.getTime + + def getLastUpdatedEpoch: Long = lastUpdated.getTime + + // These setter methods are here for Jackson, since different parts of the code try to create + // instances of this class from serialized JSON and fail if these are not present. + + private def setStartTimeEpoch(unused: Long): Unit = { } + + private def setEndTimeEpoch(unused: Long): Unit = { } + + private def setLastUpdatedEpoch(unused: Long): Unit = { } } class ExecutorStageSummary private[spark]( val taskTime : Long, val failedTasks : Int, val succeededTasks : Int, + val killedTasks : Int, val inputBytes : Long, + val inputRecords : Long, val outputBytes : Long, + val outputRecords : Long, val shuffleRead : Long, + val shuffleReadRecords : Long, val shuffleWrite : Long, + val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long) @@ -98,10 +116,12 @@ class JobData private[spark]( val numCompletedTasks: Int, val numSkippedTasks: Int, val numFailedTasks: Int, + val numKilledTasks: Int, val numActiveStages: Int, val numCompletedStages: Int, val numSkippedStages: Int, - val numFailedStages: Int) + val numFailedStages: Int, + val killedTasksSummary: Map[String, Int]) class RDDStorageInfo private[spark]( val id: Int, @@ -119,9 +139,13 @@ class RDDDataDistribution private[spark]( val memoryUsed: Long, val memoryRemaining: Long, val diskUsed: Long, + @JsonDeserialize(contentAs = classOf[JLong]) val onHeapMemoryUsed: Option[Long], + @JsonDeserialize(contentAs = classOf[JLong]) val offHeapMemoryUsed: Option[Long], + @JsonDeserialize(contentAs = classOf[JLong]) val onHeapMemoryRemaining: Option[Long], + @JsonDeserialize(contentAs = classOf[JLong]) val offHeapMemoryRemaining: Option[Long]) class RDDPartitionInfo private[spark]( @@ -135,15 +159,18 @@ class StageData private[spark]( val status: StageStatus, val stageId: Int, val attemptId: Int, + val numTasks: Int, val numActiveTasks: Int, val numCompleteTasks: Int, val numFailedTasks: Int, + val numKilledTasks: Int, val executorRunTime: Long, val executorCpuTime: Long, val submissionTime: Option[Date], val firstTaskLaunchedTime: Option[Date], val completionTime: Option[Date], + val failureReason: Option[String], val inputBytes: Long, val inputRecords: Long, @@ -157,19 +184,24 @@ class StageData private[spark]( val diskBytesSpilled: Long, val name: String, + val description: Option[String], val details: String, val schedulingPool: String, + val rddIds: Seq[Int], val accumulatorUpdates: Seq[AccumulableInfo], val tasks: Option[Map[Long, TaskData]], - val executorSummary: Option[Map[String, ExecutorStageSummary]]) + val executorSummary: Option[Map[String, ExecutorStageSummary]], + val killedTasksSummary: Map[String, Int]) class TaskData private[spark]( val taskId: Long, val index: Int, val attempt: Int, val launchTime: Date, - val duration: Option[Long] = None, + val resultFetchStart: Option[Date], + @JsonDeserialize(contentAs = classOf[JLong]) + val duration: Option[Long], val executorId: String, val host: String, val status: String, @@ -189,6 +221,7 @@ class TaskMetrics private[spark]( val resultSerializationTime: Long, val memoryBytesSpilled: Long, val diskBytesSpilled: Long, + val peakExecutionMemory: Long, val inputMetrics: InputMetrics, val outputMetrics: OutputMetrics, val shuffleReadMetrics: ShuffleReadMetrics, diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.scala b/core/src/main/scala/org/apache/spark/status/config.scala similarity index 53% rename from streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.scala rename to core/src/main/scala/org/apache/spark/status/config.scala index d3c689c790cfc..9b2385dd64f8d 100644 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.scala +++ b/core/src/main/scala/org/apache/spark/status/config.scala @@ -15,21 +15,28 @@ * limitations under the License. */ -package org.apache.spark.status.api.v1.streaming +package org.apache.spark.status -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType +import java.util.concurrent.TimeUnit -import org.apache.spark.status.api.v1.NotFoundException -import org.apache.spark.streaming.ui.StreamingJobProgressListener +import org.apache.spark.internal.config._ -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneBatchResource(listener: StreamingJobProgressListener) { +private[spark] object config { + + val LISTENER_METRICS_LOG = ConfigBuilder("spark.appStateStore.listenerMetrics.enable") + .booleanConf + .createWithDefault(false) + + val METRICS_LOG_PERIOD = ConfigBuilder("spark.appStateStore.listenerMetrics.period") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("30s") + + val MAX_CACHED_ELEMENTS = ConfigBuilder("spark.appStateStore.maxCachedElements") + .intConf + .createWithDefault(4 * 1024) + + val MAX_WRITE_QUEUE_SIZE = ConfigBuilder("spark.appStateStore.maxWriteQueueSize") + .intConf + .createWithDefault(10000) - @GET - def oneBatch(@PathParam("batchId") batchId: Long): BatchInfo = { - val someBatch = AllBatchesResource.batchInfoList(listener) - .find { _.batchId == batchId } - someBatch.getOrElse(throw new NotFoundException("unknown batch: " + batchId)) - } } diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala new file mode 100644 index 0000000000000..5c04041523b54 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -0,0 +1,205 @@ +/* + * 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.status + +import java.lang.{Integer => JInteger, Long => JLong} + +import com.fasterxml.jackson.annotation.JsonIgnore + +import org.apache.spark.kvstore.KVIndex +import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.status.api.v1._ +import org.apache.spark.status.KVUtils._ +import org.apache.spark.ui.scope._ + +private[spark] case class AppStatusStoreMetadata( + val version: Long) + +private[spark] class ApplicationInfoWrapper(val info: ApplicationInfo) { + + @JsonIgnore @KVIndex + def id: String = info.id + +} + +private[spark] class ApplicationEnvironmentInfoWrapper(val info: ApplicationEnvironmentInfo) { + + /** + * There's always a single ApplicationEnvironmentInfo object per application, so this + * ID doesn't need to be dynamic. But the KVStore API requires an ID. + */ + @JsonIgnore @KVIndex + def id: String = classOf[ApplicationEnvironmentInfoWrapper].getName() + +} + +private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) { + + @JsonIgnore @KVIndex + def id: String = info.id + + @JsonIgnore @KVIndex("active") + def active: Boolean = info.isActive + + @JsonIgnore @KVIndex("host") + def host: String = info.hostPort.split(":")(0) + +} + +/** + * Used to keep track of the stages a job was initially expected to execute, so that when the + * job finishes proper accounting of the actual number of skipped stages and tasks can be done. + */ +private[spark] class JobStageSummary( + val stageId: Int, + val attemptId: Int, + val numTasks: Int) + +/** + * Keep track of the existing stages when the job was submitted, and those that were + * completed during the job's execution. This allows a more accurate acounting of how + * many tasks were skipped for the job. + */ +private[spark] class JobDataWrapper( + val info: JobData, + val initialStages: Seq[JobStageSummary], + val submittedStages: Set[Int], + val completedStages: Set[Int]) { + + @JsonIgnore @KVIndex + def id: Int = info.jobId + +} + +private[spark] class StageDataWrapper( + val info: StageData, + val jobIds: Set[Int]) { + + @JsonIgnore @KVIndex + def id: Array[Int] = Array(info.stageId, info.attemptId) + + @JsonIgnore @KVIndex("stageId") + def stageId: Int = info.stageId + +} + +/** + * The task information is always indexed with the stage ID, since that is how the UI and API + * consume it. That means every indexed value has the stage ID and attempt ID included, aside + * from the actual data being indexed. + */ +private[spark] class TaskDataWrapper( + val info: TaskData, + val stageId: Int, + val stageAttemptId: Int) { + + @JsonIgnore @KVIndex + def id: Long = info.taskId + + @JsonIgnore @KVIndex("stage") + def stage: Array[Int] = Array(stageId, stageAttemptId) + + @JsonIgnore @KVIndex("runtime") + def runtime: Array[AnyRef] = { + val _runtime = info.taskMetrics.map(_.executorRunTime).getOrElse(-1L) + Array(stageId: JInteger, stageAttemptId: JInteger, _runtime: JLong) + } + +} + +private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) { + + @JsonIgnore @KVIndex + def id: Int = info.id + + @JsonIgnore @KVIndex("cached") + def cached: Boolean = info.numCachedPartitions > 0 + +} + +private[spark] class ExecutorStageSummaryWrapper( + val stageId: Int, + val stageAttemptId: Int, + val executorId: String, + val info: ExecutorStageSummary) { + + @JsonIgnore @KVIndex + def id: Array[Any] = Array(stageId, stageAttemptId, executorId) + + @JsonIgnore @KVIndex("stage") + def stage: Array[Int] = Array(stageId, stageAttemptId) + +} + +/** + * Store raw executor events so that the executor timeline can be drawn. The event is wrapped + * in a container so that a monotonically increasing ID can be added to it. + */ +private[spark] class ExecutorEventData( + @KVIndexParam val id: Long, + val event: SparkListenerEvent) + +private[spark] class StreamBlockData( + val name: String, + val executorId: String, + val hostPort: String, + val storageLevel: String, + val useMemory: Boolean, + val useDisk: Boolean, + val deserialized: Boolean, + val memSize: Long, + val diskSize: Long) { + + @JsonIgnore @KVIndex + def key: Array[String] = Array(name, executorId) + +} + +private[spark] class RDDOperationClusterWrapper( + val id: String, + val name: String, + val childNodes: Seq[RDDOperationNode], + val childClusters: Seq[RDDOperationClusterWrapper]) { + + def toRDDOperationCluster(): RDDOperationCluster = { + val cluster = new RDDOperationCluster(id, name) + childNodes.foreach(cluster.attachChildNode) + childClusters.foreach { child => + cluster.attachChildCluster(child.toRDDOperationCluster()) + } + cluster + } + +} + +private[spark] class RDDOperationGraphWrapper( + @KVIndexParam val stageId: Int, + val edges: Seq[RDDOperationEdge], + val outgoingEdges: Seq[RDDOperationEdge], + val incomingEdges: Seq[RDDOperationEdge], + val rootCluster: RDDOperationClusterWrapper) { + + def toRDDOperationGraph(): RDDOperationGraph = { + new RDDOperationGraph(edges, outgoingEdges, incomingEdges, rootCluster.toRDDOperationCluster()) + } + +} + +private[spark] class PoolData( + @KVIndexParam val name: String, + val stageIds: Set[Int]) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala deleted file mode 100644 index 0a14fcadf53e0..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala +++ /dev/null @@ -1,100 +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 - -import org.apache.spark.scheduler._ - -private[spark] case class BlockUIData( - blockId: BlockId, - location: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long) - -/** - * The aggregated status of stream blocks in an executor - */ -private[spark] case class ExecutorStreamBlockStatus( - executorId: String, - location: String, - blocks: Seq[BlockUIData]) { - - def totalMemSize: Long = blocks.map(_.memSize).sum - - def totalDiskSize: Long = blocks.map(_.diskSize).sum - - def numStreamBlocks: Int = blocks.size - -} - -private[spark] class BlockStatusListener extends SparkListener { - - private val blockManagers = - new mutable.HashMap[BlockManagerId, mutable.HashMap[BlockId, BlockUIData]] - - override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { - val blockId = blockUpdated.blockUpdatedInfo.blockId - if (!blockId.isInstanceOf[StreamBlockId]) { - // Now we only monitor StreamBlocks - return - } - val blockManagerId = blockUpdated.blockUpdatedInfo.blockManagerId - val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel - val memSize = blockUpdated.blockUpdatedInfo.memSize - val diskSize = blockUpdated.blockUpdatedInfo.diskSize - - synchronized { - // Drop the update info if the block manager is not registered - blockManagers.get(blockManagerId).foreach { blocksInBlockManager => - if (storageLevel.isValid) { - blocksInBlockManager.put(blockId, - BlockUIData( - blockId, - blockManagerId.hostPort, - storageLevel, - memSize, - diskSize) - ) - } else { - // If isValid is not true, it means we should drop the block. - blocksInBlockManager -= blockId - } - } - } - } - - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { - synchronized { - blockManagers.put(blockManagerAdded.blockManagerId, mutable.HashMap()) - } - } - - override def onBlockManagerRemoved( - blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = synchronized { - blockManagers -= blockManagerRemoved.blockManagerId - } - - def allExecutorStreamBlockStatus: Seq[ExecutorStreamBlockStatus] = synchronized { - blockManagers.map { case (blockManagerId, blocks) => - ExecutorStreamBlockStatus( - blockManagerId.executorId, blockManagerId.hostPort, blocks.values.toSeq) - }.toSeq - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala deleted file mode 100644 index ac60f795915a3..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ /dev/null @@ -1,111 +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 - -import org.apache.spark.SparkConf -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.scheduler._ - -/** - * :: DeveloperApi :: - * A SparkListener that maintains executor storage status. - * - * This class is thread-safe (unlike JobProgressListener) - */ -@DeveloperApi -@deprecated("This class will be removed in a future release.", "2.2.0") -class StorageStatusListener(conf: SparkConf) extends SparkListener { - // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) - private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() - private[storage] val deadExecutorStorageStatus = new mutable.ListBuffer[StorageStatus]() - private[this] val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) - - def storageStatusList: Seq[StorageStatus] = synchronized { - executorIdToStorageStatus.values.toSeq - } - - def deadStorageStatusList: Seq[StorageStatus] = synchronized { - deadExecutorStorageStatus - } - - /** Update storage status list to reflect updated block statuses */ - private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { - executorIdToStorageStatus.get(execId).foreach { storageStatus => - updatedBlocks.foreach { case (blockId, updatedStatus) => - if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.removeBlock(blockId) - } else { - storageStatus.updateBlock(blockId, updatedStatus) - } - } - } - } - - /** Update storage status list to reflect the removal of an RDD from the cache */ - private def updateStorageStatus(unpersistedRDDId: Int) { - storageStatusList.foreach { storageStatus => - storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => - storageStatus.removeBlock(blockId) - } - } - } - - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { - updateStorageStatus(unpersistRDD.rddId) - } - - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { - synchronized { - val blockManagerId = blockManagerAdded.blockManagerId - val executorId = blockManagerId.executorId - // The onHeap and offHeap memory are always defined for new applications, - // but they can be missing if we are replaying old event logs. - val storageStatus = new StorageStatus(blockManagerId, blockManagerAdded.maxMem, - blockManagerAdded.maxOnHeapMem, blockManagerAdded.maxOffHeapMem) - executorIdToStorageStatus(executorId) = storageStatus - - // Try to remove the dead storage status if same executor register the block manager twice. - deadExecutorStorageStatus.zipWithIndex.find(_._1.blockManagerId.executorId == executorId) - .foreach(toRemoveExecutor => deadExecutorStorageStatus.remove(toRemoveExecutor._2)) - } - } - - override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { - synchronized { - val executorId = blockManagerRemoved.blockManagerId.executorId - executorIdToStorageStatus.remove(executorId).foreach { status => - deadExecutorStorageStatus += status - } - if (deadExecutorStorageStatus.size > retainedDeadExecutors) { - deadExecutorStorageStatus.trimStart(1) - } - } - } - - override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { - val executorId = blockUpdated.blockUpdatedInfo.blockManagerId.executorId - val blockId = blockUpdated.blockUpdatedInfo.blockId - val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel - val memSize = blockUpdated.blockUpdatedInfo.memSize - val diskSize = blockUpdated.blockUpdatedInfo.diskSize - val blockStatus = BlockStatus(storageLevel, memSize, diskSize) - updateStorageStatus(executorId, Seq((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 bf4cf79e9faa3..2e59b9e1514de 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,37 +17,30 @@ package org.apache.spark.ui -import java.util.{Date, ServiceLoader} +import java.util.{Date, List => JList, ServiceLoader} import scala.collection.JavaConverters._ -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, - UIRoot} -import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1._ import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} -import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, JobsTab, StagesTab} -import org.apache.spark.ui.scope.RDDOperationGraphListener -import org.apache.spark.ui.storage.{StorageListener, StorageTab} +import org.apache.spark.ui.env.EnvironmentTab +import org.apache.spark.ui.exec.ExecutorsTab +import org.apache.spark.ui.jobs.{JobsTab, StagesTab} +import org.apache.spark.ui.storage.StorageTab import org.apache.spark.util.Utils /** * Top level user interface for a Spark application. */ private[spark] class SparkUI private ( + val store: AppStateStore, val sc: Option[SparkContext], val conf: SparkConf, securityManager: SecurityManager, - val environmentListener: EnvironmentListener, - val storageStatusListener: StorageStatusListener, - val executorsListener: ExecutorsListener, - val jobProgressListener: JobProgressListener, - val storageListener: StorageListener, - val operationGraphListener: RDDOperationGraphListener, var appName: String, val basePath: String, val startTime: Long) @@ -63,17 +56,18 @@ private[spark] class SparkUI private ( private var streamingJobProgressListener: Option[SparkListener] = None /** Initialize all components of the server. */ - def initialize() { - val jobsTab = new JobsTab(this) + def initialize(): Unit = { + val jobsTab = new JobsTab(this, store) attachTab(jobsTab) - val stagesTab = new StagesTab(this) + val stagesTab = new StagesTab(this, store) attachTab(stagesTab) - attachTab(new StorageTab(this)) - attachTab(new EnvironmentTab(this)) + attachTab(new StorageTab(this, store)) + attachTab(new EnvironmentTab(this, store)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) + // These should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/jobs/job/kill", "/jobs/", jobsTab.handleKillRequest, httpMethods = Set("GET", "POST"))) @@ -81,10 +75,11 @@ private[spark] class SparkUI private ( "/stages/stage/kill", "/stages/", stagesTab.handleKillRequest, httpMethods = Set("GET", "POST"))) } + initialize() def getSparkUser: String = { - environmentListener.systemProperties.toMap.getOrElse("user.name", "") + store.environmentInfo().systemProperties.toMap.getOrElse("user.name", "") } def getAppName: String = appName @@ -99,8 +94,12 @@ private[spark] class SparkUI private ( logInfo(s"Stopped Spark web UI at $webUrl") } - def getSparkUI(appId: String): Option[SparkUI] = { - if (appId == this.appId) Some(this) else None + override def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T = { + if (appId == this.appId) { + fn(this) + } else { + throw new NoSuchElementException() + } } def getApplicationInfoList: Iterator[ApplicationInfo] = { @@ -132,6 +131,7 @@ private[spark] class SparkUI private ( def setStreamingJobProgressListener(sparkListener: SparkListener): Unit = { streamingJobProgressListener = Option(sparkListener) } + } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) @@ -152,74 +152,18 @@ private[spark] object SparkUI { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) } - def createLiveUI( - sc: SparkContext, - conf: SparkConf, - listenerBus: SparkListenerBus, - jobProgressListener: JobProgressListener, - securityManager: SecurityManager, - appName: String, - startTime: Long): SparkUI = { - create(Some(sc), conf, listenerBus, securityManager, appName, - jobProgressListener = Some(jobProgressListener), startTime = startTime) - } - - def createHistoryUI( - conf: SparkConf, - listenerBus: SparkListenerBus, - securityManager: SecurityManager, - appName: String, - basePath: String, - startTime: Long): SparkUI = { - val sparkUI = create( - None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) - - val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory], - Utils.getContextOrSparkClassLoader).asScala - listenerFactories.foreach { listenerFactory => - val listeners = listenerFactory.createListeners(conf, sparkUI) - listeners.foreach(listenerBus.addListener) - } - sparkUI - } - /** - * Create a new Spark UI. - * - * @param sc optional SparkContext; this can be None when reconstituting a UI from event logs. - * @param jobProgressListener if supplied, this JobProgressListener will be used; otherwise, the - * web UI will create and register its own JobProgressListener. + * Create a new UI backed by an AppStateStore. */ - private def create( + def create( sc: Option[SparkContext], + store: AppStateStore, conf: SparkConf, - listenerBus: SparkListenerBus, securityManager: SecurityManager, appName: String, - basePath: String = "", - jobProgressListener: Option[JobProgressListener] = None, + basePath: String, startTime: Long): SparkUI = { - - val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { - val listener = new JobProgressListener(conf) - listenerBus.addListener(listener) - listener - } - - val environmentListener = new EnvironmentListener - val storageStatusListener = new StorageStatusListener(conf) - val executorsListener = new ExecutorsListener(storageStatusListener, conf) - val storageListener = new StorageListener(storageStatusListener) - val operationGraphListener = new RDDOperationGraphListener(conf) - - listenerBus.addListener(environmentListener) - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(executorsListener) - listenerBus.addListener(storageListener) - listenerBus.addListener(operationGraphListener) - - new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, - executorsListener, _jobProgressListener, storageListener, operationGraphListener, - appName, basePath, startTime) + new SparkUI(store, sc, conf, securityManager, appName, basePath, startTime) } + } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index b11f8f1555f17..17e4c37145246 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -21,22 +21,31 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.SparkConf +import org.apache.spark.status.AppStateStore +import org.apache.spark.ui._ import org.apache.spark.util.Utils -private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { - private val listener = parent.listener +private[ui] class EnvironmentPage( + parent: EnvironmentTab, + conf: SparkConf, + store: AppStateStore) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { + val appEnv = store.environmentInfo() + val jvmInformation = Map( + "Java Version" -> appEnv.runtime.javaVersion, + "Java Home" -> appEnv.runtime.javaHome, + "Scala Version" -> appEnv.runtime.scalaVersion) + val runtimeInformationTable = UIUtils.listingTable( - propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) + propertyHeader, jvmRow, jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(parent.conf, listener.sparkProperties), fixedWidth = true) - + Utils.redact(conf, appEnv.sparkProperties.toSeq), fixedWidth = true) val systemPropertiesTable = UIUtils.listingTable( - propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true) + propertyHeader, propertyRow, appEnv.systemProperties, fixedWidth = true) val classpathEntriesTable = UIUtils.listingTable( - classPathHeaders, classPathRow, listener.classpathEntries, fixedWidth = true) + classPathHeaders, classPathRow, appEnv.classpathEntries, fixedWidth = true) val content =

Runtime Information

{runtimeInformationTable} @@ -54,3 +63,9 @@ private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} } + +private[ui] class EnvironmentTab( + parent: SparkUI, + store: AppStateStore) extends SparkUITab(parent, "environment") { + attachPage(new EnvironmentPage(this, parent.conf, store)) +} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala deleted file mode 100644 index 8c18464e6477a..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ /dev/null @@ -1,51 +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.env - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.scheduler._ -import org.apache.spark.ui._ - -private[ui] class EnvironmentTab(parent: SparkUI) extends SparkUITab(parent, "environment") { - val listener = parent.environmentListener - val conf = parent.conf - attachPage(new EnvironmentPage(this)) -} - -/** - * :: DeveloperApi :: - * A SparkListener that prepares information to be displayed on the EnvironmentTab - */ -@DeveloperApi -@deprecated("This class will be removed in a future release.", "2.2.0") -class EnvironmentListener extends SparkListener { - var jvmInformation = Seq[(String, String)]() - var sparkProperties = Seq[(String, String)]() - var systemProperties = Seq[(String, String)]() - var classpathEntries = Seq[(String, String)]() - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - 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/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index 6ce3f511e89c7..483f94ed0fa05 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -22,11 +22,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Text} -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.SparkContext +import org.apache.spark.ui.{SparkUITab, UIUtils, WebUIPage} -private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage("threadDump") { - - private val sc = parent.sc +private[ui] class ExecutorThreadDumpPage( + parent: SparkUITab, + sc: Option[SparkContext]) extends WebUIPage("threadDump") { def render(request: HttpServletRequest): Seq[Node] = { val executorId = Option(request.getParameter("executorId")).map { executorId => diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala deleted file mode 100644 index b7cbed468517c..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ /dev/null @@ -1,154 +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.exec - -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.status.api.v1.{ExecutorSummary, MemoryMetrics} -import org.apache.spark.ui.{UIUtils, WebUIPage} - -// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive -private[ui] case class ExecutorSummaryInfo( - id: String, - hostPort: String, - rddBlocks: Int, - memoryUsed: Long, - diskUsed: Long, - activeTasks: Int, - failedTasks: Int, - completedTasks: Int, - totalTasks: Int, - totalDuration: Long, - totalInputBytes: Long, - totalShuffleRead: Long, - totalShuffleWrite: Long, - isBlacklisted: Int, - maxOnHeapMem: Long, - maxOffHeapMem: Long, - executorLogs: Map[String, String]) - - -private[ui] class ExecutorsPage( - parent: ExecutorsTab, - threadDumpEnabled: Boolean) - extends WebUIPage("") { - - def render(request: HttpServletRequest): Seq[Node] = { - val content = -
- { -
- - - Show Additional Metrics - - -
++ -
++ - ++ - ++ - - } -
- - UIUtils.headerSparkPage("Executors", content, parent, useDataTables = true) - } -} - -private[spark] object ExecutorsPage { - private val ON_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for on heap " + - "storage of data like RDD partitions cached in memory." - private val OFF_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for off heap " + - "storage of data like RDD partitions cached in memory." - - /** Represent an executor's info as a map given a storage status index */ - def getExecInfo( - listener: ExecutorsListener, - statusId: Int, - isActive: Boolean): ExecutorSummary = { - val status = if (isActive) { - listener.activeStorageStatusList(statusId) - } else { - listener.deadStorageStatusList(statusId) - } - val execId = status.blockManagerId.executorId - val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.numBlocks - val memUsed = status.memUsed - val maxMem = status.maxMem - val memoryMetrics = for { - onHeapUsed <- status.onHeapMemUsed - offHeapUsed <- status.offHeapMemUsed - maxOnHeap <- status.maxOnHeapMem - maxOffHeap <- status.maxOffHeapMem - } yield { - new MemoryMetrics(onHeapUsed, offHeapUsed, maxOnHeap, maxOffHeap) - } - - - val diskUsed = status.diskUsed - val taskSummary = listener.executorToTaskSummary.getOrElse(execId, ExecutorTaskSummary(execId)) - - new ExecutorSummary( - execId, - hostPort, - isActive, - rddBlocks, - memUsed, - diskUsed, - taskSummary.totalCores, - taskSummary.tasksMax, - taskSummary.tasksActive, - taskSummary.tasksFailed, - taskSummary.tasksComplete, - taskSummary.tasksActive + taskSummary.tasksFailed + taskSummary.tasksComplete, - taskSummary.duration, - taskSummary.jvmGCTime, - taskSummary.inputBytes, - taskSummary.shuffleRead, - taskSummary.shuffleWrite, - taskSummary.isBlacklisted, - maxMem, - taskSummary.executorLogs, - memoryMetrics - ) - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index aabf6e0c63c02..310d22d106773 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -17,192 +17,45 @@ package org.apache.spark.ui.exec -import scala.collection.mutable.{LinkedHashMap, ListBuffer} +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node -import org.apache.spark.{ExceptionFailure, Resubmitted, SparkConf, SparkContext} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ -import org.apache.spark.storage.{StorageStatus, StorageStatusListener} -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils, WebUIPage} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { - val listener = parent.executorsListener - val sc = parent.sc - val threadDumpEnabled = - sc.isDefined && parent.conf.getBoolean("spark.ui.threadDumpsEnabled", true) - attachPage(new ExecutorsPage(this, threadDumpEnabled)) - if (threadDumpEnabled) { - attachPage(new ExecutorThreadDumpPage(this)) - } -} + init() -private[ui] case class ExecutorTaskSummary( - var executorId: String, - var totalCores: Int = 0, - var tasksMax: Int = 0, - var tasksActive: Int = 0, - var tasksFailed: Int = 0, - var tasksComplete: Int = 0, - var duration: Long = 0L, - var jvmGCTime: Long = 0L, - var inputBytes: Long = 0L, - var inputRecords: Long = 0L, - var outputBytes: Long = 0L, - var outputRecords: Long = 0L, - var shuffleRead: Long = 0L, - var shuffleWrite: Long = 0L, - var executorLogs: Map[String, String] = Map.empty, - var isAlive: Boolean = true, - var isBlacklisted: Boolean = false -) - -/** - * :: DeveloperApi :: - * A SparkListener that prepares information to be displayed on the ExecutorsTab - */ -@DeveloperApi -@deprecated("This class will be removed in a future release.", "2.2.0") -class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: SparkConf) - extends SparkListener { - val executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]() - var executorEvents = new ListBuffer[SparkListenerEvent]() + private def init(): Unit = { + val threadDumpEnabled = + parent.sc.isDefined && parent.conf.getBoolean("spark.ui.threadDumpsEnabled", true) - private val maxTimelineExecutors = conf.getInt("spark.ui.timeline.executors.maximum", 1000) - private val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) - - def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - - def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList - - override def onExecutorAdded( - executorAdded: SparkListenerExecutorAdded): Unit = synchronized { - val eid = executorAdded.executorId - val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - taskSummary.executorLogs = executorAdded.executorInfo.logUrlMap - taskSummary.totalCores = executorAdded.executorInfo.totalCores - taskSummary.tasksMax = taskSummary.totalCores / conf.getInt("spark.task.cpus", 1) - executorEvents += executorAdded - if (executorEvents.size > maxTimelineExecutors) { - executorEvents.remove(0) + attachPage(new ExecutorsPage(this, threadDumpEnabled)) + if (threadDumpEnabled) { + attachPage(new ExecutorThreadDumpPage(this, parent.sc)) } - - val deadExecutors = executorToTaskSummary.filter(e => !e._2.isAlive) - if (deadExecutors.size > retainedDeadExecutors) { - val head = deadExecutors.head - executorToTaskSummary.remove(head._1) - } - } - - override def onExecutorRemoved( - executorRemoved: SparkListenerExecutorRemoved): Unit = synchronized { - executorEvents += executorRemoved - if (executorEvents.size > maxTimelineExecutors) { - executorEvents.remove(0) - } - executorToTaskSummary.get(executorRemoved.executorId).foreach(e => e.isAlive = false) } - override def onApplicationStart( - applicationStart: SparkListenerApplicationStart): Unit = { - applicationStart.driverLogs.foreach { logs => - val storageStatus = activeStorageStatusList.find { s => - s.blockManagerId.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER || - s.blockManagerId.executorId == SparkContext.DRIVER_IDENTIFIER - } - storageStatus.foreach { s => - val eid = s.blockManagerId.executorId - val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - taskSummary.executorLogs = logs.toMap - } - } - } - - override def onTaskStart( - taskStart: SparkListenerTaskStart): Unit = synchronized { - val eid = taskStart.taskInfo.executorId - val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - taskSummary.tasksActive += 1 - } - - override def onTaskEnd( - taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = info.executorId - val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - taskEnd.reason match { - case Resubmitted => - // Note: For resubmitted tasks, we continue to use the metrics that belong to the - // first attempt of this task. This may not be 100% accurate because the first attempt - // could have failed half-way through. The correct fix would be to keep track of the - // metrics added by each attempt, but this is much more complicated. - return - case _: ExceptionFailure => - taskSummary.tasksFailed += 1 - case _ => - taskSummary.tasksComplete += 1 - } - if (taskSummary.tasksActive >= 1) { - taskSummary.tasksActive -= 1 - } - taskSummary.duration += info.duration - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - taskSummary.inputBytes += metrics.inputMetrics.bytesRead - taskSummary.inputRecords += metrics.inputMetrics.recordsRead - taskSummary.outputBytes += metrics.outputMetrics.bytesWritten - taskSummary.outputRecords += metrics.outputMetrics.recordsWritten - - taskSummary.shuffleRead += metrics.shuffleReadMetrics.remoteBytesRead - taskSummary.shuffleWrite += metrics.shuffleWriteMetrics.bytesWritten - taskSummary.jvmGCTime += metrics.jvmGCTime - } - } - } - - private def updateExecutorBlacklist( - eid: String, - isBlacklisted: Boolean): Unit = { - val execTaskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - execTaskSummary.isBlacklisted = isBlacklisted - } - - override def onExecutorBlacklisted( - executorBlacklisted: SparkListenerExecutorBlacklisted) - : Unit = synchronized { - updateExecutorBlacklist(executorBlacklisted.executorId, true) - } - - override def onExecutorUnblacklisted( - executorUnblacklisted: SparkListenerExecutorUnblacklisted) - : Unit = synchronized { - updateExecutorBlacklist(executorUnblacklisted.executorId, false) - } - - override def onNodeBlacklisted( - nodeBlacklisted: SparkListenerNodeBlacklisted) - : Unit = synchronized { - // Implicitly blacklist every executor associated with this node, and show this in the UI. - activeStorageStatusList.foreach { status => - if (status.blockManagerId.host == nodeBlacklisted.hostId) { - updateExecutorBlacklist(status.blockManagerId.executorId, true) - } - } - } +} - override def onNodeUnblacklisted( - nodeUnblacklisted: SparkListenerNodeUnblacklisted) - : Unit = synchronized { - // Implicitly unblacklist every executor associated with this node, regardless of how - // they may have been blacklisted initially (either explicitly through executor blacklisting - // or implicitly through node blacklisting). Show this in the UI. - activeStorageStatusList.foreach { status => - if (status.blockManagerId.host == nodeUnblacklisted.hostId) { - updateExecutorBlacklist(status.blockManagerId.executorId, false) - } - } +private[ui] class ExecutorsPage( + parent: SparkUITab, + threadDumpEnabled: Boolean) + extends WebUIPage("") { + + def render(request: HttpServletRequest): Seq[Node] = { + val content = +
+ { +
++ + ++ + ++ + + } +
+ + UIUtils.headerSparkPage("Executors", content, parent, useDataTables = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 18be0870746e9..d2175f3a2308e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -23,18 +23,20 @@ import javax.servlet.http.HttpServletRequest import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} +import scala.util.Try import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler._ +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1 import org.apache.spark.ui._ -import org.apache.spark.ui.jobs.UIData.{JobUIData, StageUIData} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished jobs */ -private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { +private[ui] class AllJobsPage(parent: JobsTab, store: AppStateStore) extends WebUIPage("") { private val JOBS_LEGEND =
Removed
.toString.filter(_ != '\n') - private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { - val lastStageInfo = Option(job.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => parent.jobProgresslistener.stageIdToInfo.get(ids.max)} - val lastStageData = lastStageInfo.flatMap { s => - parent.jobProgresslistener.stageIdToData.get((s.stageId, s.attemptId)) - } - val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val description = lastStageData.flatMap(_.description).getOrElse("") - (name, description) - } - - private def makeJobEvent(jobUIDatas: Seq[JobUIData]): Seq[String] = { - jobUIDatas.filter { jobUIData => - jobUIData.status != JobExecutionStatus.UNKNOWN && jobUIData.submissionTime.isDefined - }.map { jobUIData => - val jobId = jobUIData.jobId - val status = jobUIData.status - val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData) + private def makeJobEvent(jobs: Seq[v1.JobData]): Seq[String] = { + jobs.filter { job => + job.status != JobExecutionStatus.UNKNOWN && job.submissionTime.isDefined + }.map { job => + val jobId = job.jobId + val status = job.status val displayJobDescription = - if (jobDescription.isEmpty) { - jobName + if (job.description.isEmpty) { + job.name } else { - UIUtils.makeDescription(jobDescription, "", plainText = true).text + UIUtils.makeDescription(job.description.get, "", plainText = true).text } - val submissionTime = jobUIData.submissionTime.get - val completionTimeOpt = jobUIData.completionTime - val completionTime = completionTimeOpt.getOrElse(System.currentTimeMillis()) + val submissionTime = job.submissionTime.get.getTime() + val completionTime = job.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) val classNameByStatus = status match { case JobExecutionStatus.SUCCEEDED => "succeeded" case JobExecutionStatus.FAILED => "failed" @@ -170,7 +158,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } private def makeTimeline( - jobs: Seq[JobUIData], + jobs: Seq[v1.JobData], executors: Seq[SparkListenerEvent], startTime: Long): Seq[Node] = { @@ -218,7 +206,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { request: HttpServletRequest, tableHeaderId: String, jobTag: String, - jobs: Seq[JobUIData], + jobs: Seq[v1.JobData], killEnabled: Boolean): Seq[Node] = { val allParameters = request.getParameterMap.asScala.toMap val parameterOtherTable = allParameters.filterNot(_._1.startsWith(jobTag)) @@ -257,14 +245,13 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { try { new JobPagedTable( + store, jobs, tableHeaderId, jobTag, UIUtils.prependBaseUri(parent.basePath), "jobs", // subPath parameterOtherTable, - parent.jobProgresslistener.stageIdToInfo, - parent.jobProgresslistener.stageIdToData, killEnabled, currentTime, jobIdTitle, @@ -284,107 +271,120 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def render(request: HttpServletRequest): Seq[Node] = { - val listener = parent.jobProgresslistener - listener.synchronized { - val startTime = listener.startTime - val endTime = listener.endTime - val activeJobs = listener.activeJobs.values.toSeq - val completedJobs = listener.completedJobs.reverse - val failedJobs = listener.failedJobs.reverse - - val activeJobsTable = - jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) - val completedJobsTable = - jobsTable(request, "completed", "completedJob", completedJobs, killEnabled = false) - val failedJobsTable = - jobsTable(request, "failed", "failedJob", failedJobs, killEnabled = false) - - val shouldShowActiveJobs = activeJobs.nonEmpty - val shouldShowCompletedJobs = completedJobs.nonEmpty - val shouldShowFailedJobs = failedJobs.nonEmpty - - val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) { - s"${completedJobs.size}" - } else { - s"${listener.numCompletedJobs}, only showing ${completedJobs.size}" + val appInfo = store.appInfo() + val startTime = appInfo.attempts.head.startTime.getTime() + val endTime = appInfo.attempts.head.endTime.getTime() + + val activeJobs = new ListBuffer[v1.JobData]() + val _completedJobs = new ListBuffer[v1.JobData]() + val _failedJobs = new ListBuffer[v1.JobData]() + + store.jobsList(null).foreach { job => + job.status match { + case JobExecutionStatus.SUCCEEDED => + _completedJobs += job + case JobExecutionStatus.FAILED => + _failedJobs += job + case _ => + activeJobs += job } + } - val summary: NodeSeq = -
-
    -
  • - User: - {parent.getSparkUser} -
  • -
  • - Total Uptime: - { - if (endTime < 0 && parent.sc.isDefined) { - UIUtils.formatDuration(System.currentTimeMillis() - startTime) - } else if (endTime > 0) { - UIUtils.formatDuration(endTime - startTime) - } - } -
  • -
  • - Scheduling Mode: - {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} -
  • + val completedJobs = _completedJobs.toSeq.reverse + val failedJobs = _failedJobs.toSeq.reverse + + val activeJobsTable = + jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) + val completedJobsTable = + jobsTable(request, "completed", "completedJob", completedJobs, killEnabled = false) + val failedJobsTable = + jobsTable(request, "failed", "failedJob", failedJobs, killEnabled = false) + + val shouldShowActiveJobs = activeJobs.nonEmpty + val shouldShowCompletedJobs = completedJobs.nonEmpty + val shouldShowFailedJobs = failedJobs.nonEmpty + + val completedJobNumStr = s"${completedJobs.size}" + val schedulingMode = store.environmentInfo().sparkProperties.toMap + .get("spark.scheduler.mode") + .map { mode => SchedulingMode.withName(mode).toString } + .getOrElse("Unknown") + + val summary: NodeSeq = +
    +
      +
    • + User: + {parent.getSparkUser} +
    • +
    • + Total Uptime: { - if (shouldShowActiveJobs) { -
    • - Active Jobs: - {activeJobs.size} -
    • + if (endTime < 0 && parent.sc.isDefined) { + UIUtils.formatDuration(System.currentTimeMillis() - startTime) + } else if (endTime > 0) { + UIUtils.formatDuration(endTime - startTime) } } - { - if (shouldShowCompletedJobs) { -
    • - Completed Jobs: - {completedJobNumStr} -
    • - } + +
    • + Scheduling Mode: + {schedulingMode} +
    • + { + if (shouldShowActiveJobs) { +
    • + Active Jobs: + {activeJobs.size} +
    • } - { - if (shouldShowFailedJobs) { -
    • - Failed Jobs: - {listener.numFailedJobs} -
    • - } + } + { + if (shouldShowCompletedJobs) { +
    • + Completed Jobs: + {completedJobNumStr} +
    • } -
    -
    + } + { + if (shouldShowFailedJobs) { +
  • + Failed Jobs: + {failedJobs.size} +
  • + } + } +
+
- var content = summary - val executorListener = parent.executorListener - content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, - executorListener.executorEvents, startTime) + var content = summary + content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, + store.executorEvents(), startTime) - if (shouldShowActiveJobs) { - content ++=

Active Jobs ({activeJobs.size})

++ - activeJobsTable - } - if (shouldShowCompletedJobs) { - content ++=

Completed Jobs ({completedJobNumStr})

++ - completedJobsTable - } - if (shouldShowFailedJobs) { - content ++=

Failed Jobs ({failedJobs.size})

++ - failedJobsTable - } + if (shouldShowActiveJobs) { + content ++=

Active Jobs ({activeJobs.size})

++ + activeJobsTable + } + if (shouldShowCompletedJobs) { + content ++=

Completed Jobs ({completedJobNumStr})

++ + completedJobsTable + } + if (shouldShowFailedJobs) { + content ++=

Failed Jobs ({failedJobs.size})

++ + failedJobsTable + } - val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + - " Click on a job to see information about the stages of tasks inside it." + val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + + " Click on a job to see information about the stages of tasks inside it." - UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) - } + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) } + } private[ui] class JobTableRowData( - val jobData: JobUIData, + val jobData: v1.JobData, val lastStageName: String, val lastStageDescription: String, val duration: Long, @@ -395,9 +395,8 @@ private[ui] class JobTableRowData( val detailUrl: String) private[ui] class JobDataSource( - jobs: Seq[JobUIData], - stageIdToInfo: HashMap[Int, StageInfo], - stageIdToData: HashMap[(Int, Int), StageUIData], + store: AppStateStore, + jobs: Seq[v1.JobData], basePath: String, currentTime: Long, pageSize: Int, @@ -418,40 +417,28 @@ private[ui] class JobDataSource( r } - private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { - val lastStageInfo = Option(job.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => stageIdToInfo.get(ids.max)} - val lastStageData = lastStageInfo.flatMap { s => - stageIdToData.get((s.stageId, s.attemptId)) - } - val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val description = lastStageData.flatMap(_.description).getOrElse("") - (name, description) - } - - private def jobRow(jobData: JobUIData): JobTableRowData = { - val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(jobData) + private def jobRow(jobData: v1.JobData): JobTableRowData = { val duration: Option[Long] = { jobData.submissionTime.map { start => - val end = jobData.completionTime.getOrElse(System.currentTimeMillis()) - end - start + val end = jobData.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) + end - start.getTime() } } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val submissionTime = jobData.submissionTime val formattedSubmissionTime = submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") - val jobDescription = UIUtils.makeDescription(lastStageDescription, basePath, plainText = false) + val jobDescription = UIUtils.makeDescription(jobData.description.getOrElse(""), + basePath, plainText = false) val detailUrl = "%s/jobs/job?id=%s".format(basePath, jobData.jobId) - new JobTableRowData ( + new JobTableRowData( jobData, - lastStageName, - lastStageDescription, + jobData.name, + jobData.description.getOrElse(jobData.name), duration.getOrElse(-1), formattedDuration, - submissionTime.getOrElse(-1), + submissionTime.map(_.getTime()).getOrElse(-1L), formattedSubmissionTime, jobDescription, detailUrl @@ -480,14 +467,13 @@ private[ui] class JobDataSource( } private[ui] class JobPagedTable( - data: Seq[JobUIData], + store: AppStateStore, + data: Seq[v1.JobData], tableHeaderId: String, jobTag: String, basePath: String, subPath: String, parameterOtherTable: Iterable[String], - stageIdToInfo: HashMap[Int, StageInfo], - stageIdToData: HashMap[(Int, Int), StageUIData], killEnabled: Boolean, currentTime: Long, jobIdTitle: String, @@ -510,9 +496,8 @@ private[ui] class JobPagedTable( override def pageNumberFormField: String = jobTag + ".page" override val dataSource = new JobDataSource( + store, data, - stageIdToInfo, - stageIdToData, basePath, currentTime, pageSize, @@ -624,14 +609,14 @@ private[ui] class JobPagedTable( {jobTableRow.formattedDuration} - {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {job.numCompletedStages}/{job.stageIds.size - job.numSkippedStages} {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, failed = job.numFailedTasks, skipped = job.numSkippedTasks, - reasonToNumKilled = job.reasonToNumKilled, total = job.numTasks - job.numSkippedTasks)} + reasonToNumKilled = job.killedTasksSummary, total = job.numTasks - job.numSkippedTasks)} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 2b0816e35747d..8eeb9a2026aaa 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -22,120 +22,125 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1._ import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc - private val listener = parent.progressListener private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { - listener.synchronized { - val activeStages = listener.activeStages.values.toSeq - val pendingStages = listener.pendingStages.values.toSeq - val completedStages = listener.completedStages.reverse - val numCompletedStages = listener.numCompletedStages - val failedStages = listener.failedStages.reverse - val numFailedStages = listener.numFailedStages - val subPath = "stages" + val allStages = parent.store.stageList(null) - val activeStagesTable = - new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, - killEnabled = parent.killEnabled, isFailedStage = false) - val pendingStagesTable = - new StageTableBase(request, pendingStages, "pending", "pendingStage", parent.basePath, - subPath, parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val completedStagesTable = - new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, - subPath, parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val failedStagesTable = - new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = true) + val activeStages = allStages.filter(_.status == StageStatus.ACTIVE) + val pendingStages = allStages.filter(_.status == StageStatus.PENDING) + val completedStages = allStages.filter(_.status == StageStatus.COMPLETE) + val failedStages = allStages.filter(_.status == StageStatus.FAILED).reverse - // For now, pool information is only accessible in live UIs - val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) - val poolTable = new PoolTable(pools, parent) + val numCompletedStages = completedStages.size + val numFailedStages = failedStages.size + val subPath = "stages" - val shouldShowActiveStages = activeStages.nonEmpty - val shouldShowPendingStages = pendingStages.nonEmpty - val shouldShowCompletedStages = completedStages.nonEmpty - val shouldShowFailedStages = failedStages.nonEmpty + val activeStagesTable = + new StageTableBase(parent.store, request, activeStages, "active", "activeStage", + parent.basePath, subPath, parent.isFairScheduler, parent.killEnabled, false) + val pendingStagesTable = + new StageTableBase(parent.store, request, pendingStages, "pending", "pendingStage", + parent.basePath, subPath, parent.isFairScheduler, false, false) + val completedStagesTable = + new StageTableBase(parent.store, request, completedStages, "completed", "completedStage", + parent.basePath, subPath, parent.isFairScheduler, false, false) + val failedStagesTable = + new StageTableBase(parent.store, request, failedStages, "failed", "failedStage", + parent.basePath, subPath, parent.isFairScheduler, false, true) - val completedStageNumStr = if (numCompletedStages == completedStages.size) { - s"$numCompletedStages" - } else { - s"$numCompletedStages, only showing ${completedStages.size}" + // For now, pool information is only accessible in live UIs + val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]).map { pool => + val uiPool = try { + parent.store.pool(pool.name) + } catch { + case _: NoSuchElementException => + new PoolData(pool.name, Set()) } + pool -> uiPool + }.toMap + val poolTable = new PoolTable(pools, parent) - val summary: NodeSeq = -
-
    - { - if (shouldShowActiveStages) { -
  • - Active Stages: - {activeStages.size} -
  • - } + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = pendingStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty + + val completedStageNumStr = if (numCompletedStages == completedStages.size) { + s"$numCompletedStages" + } else { + s"$numCompletedStages, only showing ${completedStages.size}" + } + + val summary: NodeSeq = +
    +
      + { + if (shouldShowActiveStages) { +
    • + Active Stages: + {activeStages.size} +
    • } - { - if (shouldShowPendingStages) { -
    • - Pending Stages: - {pendingStages.size} -
    • - } + } + { + if (shouldShowPendingStages) { +
    • + Pending Stages: + {pendingStages.size} +
    • } - { - if (shouldShowCompletedStages) { -
    • - Completed Stages: - {completedStageNumStr} -
    • - } + } + { + if (shouldShowCompletedStages) { +
    • + Completed Stages: + {completedStageNumStr} +
    • } - { - if (shouldShowFailedStages) { -
    • - Failed Stages: - {numFailedStages} -
    • - } + } + { + if (shouldShowFailedStages) { +
    • + Failed Stages: + {numFailedStages} +
    • } -
    -
    - - var content = summary ++ - { - if (sc.isDefined && isFairScheduler) { -

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq - } else { - Seq[Node]() } +
+
+ + var content = summary ++ + { + if (sc.isDefined && isFairScheduler) { +

{pools.size} Fair Scheduler Pools

++ poolTable.toNodeSeq + } else { + Seq[Node]() } - if (shouldShowActiveStages) { - content ++=

Active Stages ({activeStages.size})

++ - activeStagesTable.toNodeSeq } - if (shouldShowPendingStages) { - content ++=

Pending Stages ({pendingStages.size})

++ - pendingStagesTable.toNodeSeq - } - if (shouldShowCompletedStages) { - content ++=

Completed Stages ({completedStageNumStr})

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

Failed Stages ({numFailedStages})

++ - failedStagesTable.toNodeSeq - } - UIUtils.headerSparkPage("Stages for All Jobs", content, parent) + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingStages.size})

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

Completed Stages ({completedStageNumStr})

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

Failed Stages ({numFailedStages})

++ + failedStagesTable.toNodeSeq + } + UIUtils.headerSparkPage("Stages for All Jobs", content, parent) } } - 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 382a6f979f2e6..a82f591d79826 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,37 +17,36 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException + import scala.collection.mutable import scala.xml.{Node, Unparsed} +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1.StageData import org.apache.spark.ui.{ToolTips, UIUtils} -import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { - private val listener = parent.progressListener +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, store: AppStateStore) { def toNodeSeq: Seq[Node] = { - listener.synchronized { - executorTable() + val stageData = try { + Some(store.stageAttempt(stageId, stageAttemptId)) + } catch { + case _: NoSuchElementException => None } - } - - /** Special table which merges two header cells. */ - private def executorTable[T](): Seq[Node] = { - val stageData = listener.stageIdToData.get((stageId, stageAttemptId)) var hasInput = false var hasOutput = false var hasShuffleWrite = false var hasShuffleRead = false var hasBytesSpilled = false stageData.foreach { data => - hasInput = data.hasInput - hasOutput = data.hasOutput - hasShuffleRead = data.hasShuffleRead - hasShuffleWrite = data.hasShuffleWrite - hasBytesSpilled = data.hasBytesSpilled + hasInput = data.inputBytes > 0 + hasOutput = data.outputBytes > 0 + hasShuffleRead = data.shuffleReadBytes > 0 + hasShuffleWrite = data.shuffleWriteBytes > 0 + hasBytesSpilled = (data.memoryBytesSpilled + data.diskBytesSpilled) > 0 } @@ -92,7 +91,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage - {createExecutorTable()} + {createExecutorTable(stageData)}
} - private def createExecutorTable() : Seq[Node] = { - // Make an executor-id -> address map - val executorIdToAddress = mutable.HashMap[String, String]() - listener.blockManagerIds.foreach { blockManagerId => - val address = blockManagerId.hostPort - val executorId = blockManagerId.executorId - executorIdToAddress.put(executorId, address) - } - - listener.stageIdToData.get((stageId, stageAttemptId)) match { - case Some(stageData: StageUIData) => - stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => + private def createExecutorTable(stageDataOpt: Option[StageData]) : Seq[Node] = { + stageDataOpt match { + case Some(stageData) => + stageData.executorSummary.getOrElse(Map()).toSeq.sortBy(_._1).map { case (k, v) => + val executor = store.executorSummary(k)
{k}
{ - val logs = parent.executorsListener.executorToTaskSummary.get(k) - .map(_.executorLogs).getOrElse(Map.empty) - logs.map { + executor.map(_.executorLogs).getOrElse(Map.empty).map { case (logName, logUrl) => } }
- {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} + {executor.map { e => e.hostPort }.getOrElse(k, "CANNOT FIND ADDRESS")} {UIUtils.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks + v.reasonToNumKilled.values.sum} + {v.failedTasks + v.succeededTasks + v.killedTasks} {v.failedTasks} - {v.reasonToNumKilled.values.sum} + {v.killedTasks} {v.succeededTasks} - {if (stageData.hasInput) { + {if (stageData.inputBytes > 0) { {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} }} - {if (stageData.hasOutput) { + {if (stageData.outputBytes > 0) { {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} }} - {if (stageData.hasShuffleRead) { + {if (stageData.shuffleReadBytes > 0) { {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} }} - {if (stageData.hasShuffleWrite) { + {if (stageData.shuffleWriteBytes > 0) { {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} }} - {if (stageData.hasBytesSpilled) { + {if ((stageData.memoryBytesSpilled + stageData.diskBytesSpilled) > 0) { {Utils.bytesToString(v.memoryBytesSpilled)} @@ -165,7 +155,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage {Utils.bytesToString(v.diskBytesSpilled)} }} - {v.isBlacklisted} + {executor.map(_.isBlacklisted).getOrElse(false)} } case None => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 3131c4a1eb7d4..976fd4e611d06 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -21,16 +21,19 @@ import java.util.{Date, Locale} import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{Buffer, ListBuffer} +import scala.util.Try import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler._ -import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1 +import org.apache.spark.ui._ /** Page showing statistics and stage list for a given job */ -private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { +private[ui] class JobPage(parent: JobsTab, store: AppStateStore) extends WebUIPage("job") { private val STAGES_LEGEND =
@@ -55,14 +58,15 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { Removed
.toString.filter(_ != '\n') - private def makeStageEvent(stageInfos: Seq[StageInfo]): Seq[String] = { + private def makeStageEvent(stageInfos: Seq[v1.StageData]): Seq[String] = { stageInfos.map { stage => val stageId = stage.stageId val attemptId = stage.attemptId val name = stage.name - val status = stage.getStatusString - val submissionTime = stage.submissionTime.get - val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) + val status = stage.status.toString + val submissionTime = stage.submissionTime.get.getTime() + val completionTime = stage.completionTime.map(_.getTime()) + .getOrElse(System.currentTimeMillis()) // The timeline library treats contents as HTML, so we have to escape them. We need to add // extra layers of escaping in order to embed this in a Javascript string literal. @@ -78,10 +82,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'data-placement="top" data-html="true"' + | 'data-title="${jsEscapedName} (Stage ${stageId}.${attemptId})
' + | 'Status: ${status.toUpperCase(Locale.ROOT)}
' + - | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + + | 'Submitted: ${UIUtils.formatDate(submissionTime)}' + | '${ if (status != "running") { - s"""
Completed: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
Completed: ${UIUtils.formatDate(completionTime)}""" } else { "" } @@ -139,7 +143,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } private def makeTimeline( - stages: Seq[StageInfo], + stages: Seq[v1.StageData], executors: Seq[SparkListenerEvent], appStartTime: Long): Seq[Node] = { @@ -184,173 +188,180 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } def render(request: HttpServletRequest): Seq[Node] = { - val listener = parent.jobProgresslistener + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - listener.synchronized { - val parameterId = request.getParameter("id") - require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - - val jobId = parameterId.toInt - val jobDataOption = listener.jobIdToData.get(jobId) - if (jobDataOption.isEmpty) { - val content = -
-

No information to display for job {jobId}

-
- return UIUtils.headerSparkPage( - s"Details for Job $jobId", content, parent) - } - val jobData = jobDataOption.get - val isComplete = jobData.status != JobExecutionStatus.RUNNING - val stages = jobData.stageIds.map { stageId => - // This could be empty if the JobProgressListener hasn't received information about the - // stage or if the stage information has been garbage collected - listener.stageIdToInfo.getOrElse(stageId, - new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown")) + val jobId = parameterId.toInt + val jobDataOption = Try(store.job(jobId)).toOption + if (jobDataOption.isEmpty) { + val content = +
+

No information to display for job {jobId}

+
+ return UIUtils.headerSparkPage( + s"Details for Job $jobId", content, parent) + } + val jobData = jobDataOption.get + val isComplete = jobData.status != JobExecutionStatus.RUNNING + val stages = jobData.stageIds.map { stageId => + // This could be empty if the listener hasn't received information about the + // stage or if the stage information has been garbage collected + store.stageData(stageId).lastOption.getOrElse { + new v1.StageData( + v1.StageStatus.PENDING, + stageId, + 0, 0, 0, 0, 0, 0, 0L, 0L, None, None, None, None, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, + "Unknown", + None, + "Unknown", + null, + Nil, + Nil, + None, + None, + Map()) } + } - val activeStages = Buffer[StageInfo]() - val completedStages = Buffer[StageInfo]() - // If the job is completed, then any pending stages are displayed as "skipped": - val pendingOrSkippedStages = Buffer[StageInfo]() - val failedStages = Buffer[StageInfo]() - for (stage <- stages) { - if (stage.submissionTime.isEmpty) { - pendingOrSkippedStages += stage - } else if (stage.completionTime.isDefined) { - if (stage.failureReason.isDefined) { - failedStages += stage - } else { - completedStages += stage - } + val activeStages = Buffer[v1.StageData]() + val completedStages = Buffer[v1.StageData]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = Buffer[v1.StageData]() + val failedStages = Buffer[v1.StageData]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingOrSkippedStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.status == v1.StageStatus.FAILED) { + failedStages += stage } else { - activeStages += stage + completedStages += stage } + } else { + activeStages += stage } + } - val basePath = "jobs/job" + val basePath = "jobs/job" - val pendingOrSkippedTableId = - if (isComplete) { - "pending" - } else { - "skipped" - } + val pendingOrSkippedTableId = + if (isComplete) { + "pending" + } else { + "skipped" + } - val activeStagesTable = - new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = parent.killEnabled, isFailedStage = false) - val pendingOrSkippedStagesTable = - new StageTableBase(request, pendingOrSkippedStages, pendingOrSkippedTableId, "pendingStage", - parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val completedStagesTable = - new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val failedStagesTable = - new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = true) + val activeStagesTable = + new StageTableBase(store, request, activeStages, "active", "activeStage", parent.basePath, + basePath, parent.isFairScheduler, + killEnabled = parent.killEnabled, isFailedStage = false) + val pendingOrSkippedStagesTable = + new StageTableBase(store, request, pendingOrSkippedStages, pendingOrSkippedTableId, + "pendingStage", parent.basePath, basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) + val completedStagesTable = + new StageTableBase(store, request, completedStages, "completed", "completedStage", + parent.basePath, basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) + val failedStagesTable = + new StageTableBase(store, request, failedStages, "failed", "failedStage", parent.basePath, + basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = true) - val shouldShowActiveStages = activeStages.nonEmpty - val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty - val shouldShowCompletedStages = completedStages.nonEmpty - val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty - val shouldShowFailedStages = failedStages.nonEmpty + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty - val summary: NodeSeq = -
-
    -
  • - Status: - {jobData.status} -
  • - { - if (jobData.jobGroup.isDefined) { -
  • - Job Group: - {jobData.jobGroup.get} -
  • - } - } - { - if (shouldShowActiveStages) { -
  • - Active Stages: - {activeStages.size} -
  • - } - } - { - if (shouldShowPendingStages) { -
  • - - Pending Stages: - {pendingOrSkippedStages.size} -
  • - } + val summary: NodeSeq = +
    +
      +
    • + Status: + {jobData.status} +
    • + { + if (jobData.jobGroup.isDefined) { +
    • + Job Group: + {jobData.jobGroup.get} +
    • } - { - if (shouldShowCompletedStages) { -
    • - Completed Stages: - {completedStages.size} -
    • - } + } + { + if (shouldShowActiveStages) { +
    • + Active Stages: + {activeStages.size} +
    • } - { - if (shouldShowSkippedStages) { + } + { + if (shouldShowPendingStages) {
    • - Skipped Stages: - {pendingOrSkippedStages.size} + + Pending Stages: + {pendingOrSkippedStages.size}
    • } + } + { + if (shouldShowCompletedStages) { +
    • + Completed Stages: + {completedStages.size} +
    • } - { - if (shouldShowFailedStages) { -
    • - Failed Stages: - {failedStages.size} -
    • - } + } + { + if (shouldShowSkippedStages) { +
    • + Skipped Stages: + {pendingOrSkippedStages.size} +
    • + } + } + { + if (shouldShowFailedStages) { +
    • + Failed Stages: + {failedStages.size} +
    • } -
    -
    + } +
+
- var content = summary - val appStartTime = listener.startTime - val executorListener = parent.executorListener - val operationGraphListener = parent.operationGraphListener + var content = summary + val appStartTime = store.appInfo().attempts.head.startTime.getTime() - content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, - executorListener.executorEvents, appStartTime) + content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, + store.executorEvents(), appStartTime) - content ++= UIUtils.showDagVizForJob( - jobId, operationGraphListener.getOperationGraphForJob(jobId)) + content ++= UIUtils.showDagVizForJob( + jobId, store.operationGraphForJob(jobId)) - if (shouldShowActiveStages) { - content ++=

Active Stages ({activeStages.size})

++ - activeStagesTable.toNodeSeq - } - if (shouldShowPendingStages) { - content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ - pendingOrSkippedStagesTable.toNodeSeq - } - if (shouldShowCompletedStages) { - content ++=

Completed Stages ({completedStages.size})

++ - completedStagesTable.toNodeSeq - } - if (shouldShowSkippedStages) { - content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ - pendingOrSkippedStagesTable.toNodeSeq - } - if (shouldShowFailedStages) { - content ++=

Failed Stages ({failedStages.size})

++ - failedStagesTable.toNodeSeq - } - UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true) + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingOrSkippedStages.size})

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

Completed Stages ({completedStages.size})

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

Skipped Stages ({pendingOrSkippedStages.size})

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

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq } + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true) } } 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 deleted file mode 100644 index 8870187f2219c..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ /dev/null @@ -1,599 +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.jobs - -import java.util.concurrent.TimeoutException - -import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap, ListBuffer} - -import org.apache.spark._ -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import org.apache.spark.storage.BlockManagerId -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData._ - -/** - * :: DeveloperApi :: - * 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 EventBus loop may otherwise be reading and - * updating the internal data structures concurrently. - */ -@DeveloperApi -@deprecated("This class will be removed in a future release.", "2.2.0") -class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { - - // Define a handful of type aliases so that data structures' types can serve as documentation. - // These type aliases are public because they're used in the types of public fields: - - type JobId = Int - type JobGroupId = String - type StageId = Int - type StageAttemptId = Int - type PoolName = String - type ExecutorId = String - - // Application: - @volatile var startTime = -1L - @volatile var endTime = -1L - - // Jobs: - val activeJobs = new HashMap[JobId, JobUIData] - val completedJobs = ListBuffer[JobUIData]() - val failedJobs = ListBuffer[JobUIData]() - val jobIdToData = new HashMap[JobId, JobUIData] - val jobGroupToJobIds = new HashMap[JobGroupId, HashSet[JobId]] - - // Stages: - val pendingStages = new HashMap[StageId, StageInfo] - val activeStages = new HashMap[StageId, StageInfo] - val completedStages = ListBuffer[StageInfo]() - val skippedStages = ListBuffer[StageInfo]() - val failedStages = ListBuffer[StageInfo]() - val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] - val stageIdToInfo = new HashMap[StageId, StageInfo] - val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]] - val poolToActiveStages = HashMap[PoolName, HashMap[StageId, StageInfo]]() - // Total of completed and failed stages that have ever been run. These may be greater than - // `completedStages.size` and `failedStages.size` if we have run more stages or jobs than - // JobProgressListener's retention limits. - var numCompletedStages = 0 - var numFailedStages = 0 - var numCompletedJobs = 0 - var numFailedJobs = 0 - - // Misc: - val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]() - - def blockManagerIds: Seq[BlockManagerId] = executorIdToBlockManagerId.values.toSeq - - var schedulingMode: Option[SchedulingMode] = None - - // To limit the total memory usage of JobProgressListener, we only track information for a fixed - // number of non-active jobs and stages (there is no limit for active jobs and stages): - - val retainedStages = conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) - val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) - val retainedTasks = conf.get(UI_RETAINED_TASKS) - - // We can test for memory leaks by ensuring that collections that track non-active jobs and - // stages do not grow without bound and that collections for active jobs/stages eventually become - // empty once Spark is idle. Let's partition our collections into ones that should be empty - // once Spark is idle and ones that should have a hard- or soft-limited sizes. - // These methods are used by unit tests, but they're defined here so that people don't forget to - // update the tests when adding new collections. Some collections have multiple levels of - // nesting, etc, so this lets us customize our notion of "size" for each structure: - - // These collections should all be empty once Spark is idle (no active stages / jobs): - private[spark] def getSizesOfActiveStateTrackingCollections: Map[String, Int] = { - Map( - "activeStages" -> activeStages.size, - "activeJobs" -> activeJobs.size, - "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum, - "stageIdToActiveJobIds" -> stageIdToActiveJobIds.values.map(_.size).sum - ) - } - - // These collections should stop growing once we have run at least `spark.ui.retainedStages` - // stages and `spark.ui.retainedJobs` jobs: - private[spark] def getSizesOfHardSizeLimitedCollections: Map[String, Int] = { - Map( - "completedJobs" -> completedJobs.size, - "failedJobs" -> failedJobs.size, - "completedStages" -> completedStages.size, - "skippedStages" -> skippedStages.size, - "failedStages" -> failedStages.size - ) - } - - // These collections may grow arbitrarily, but once Spark becomes idle they should shrink back to - // some bound based on the `spark.ui.retainedStages` and `spark.ui.retainedJobs` settings: - private[spark] def getSizesOfSoftSizeLimitedCollections: Map[String, Int] = { - Map( - "jobIdToData" -> jobIdToData.size, - "stageIdToData" -> stageIdToData.size, - "stageIdToStageInfo" -> stageIdToInfo.size, - "jobGroupToJobIds" -> jobGroupToJobIds.values.map(_.size).sum, - // Since jobGroupToJobIds is map of sets, check that we don't leak keys with empty values: - "jobGroupToJobIds keySet" -> jobGroupToJobIds.keys.size - ) - } - - /** If stages is too large, remove and garbage collect old stages */ - private def trimStagesIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { - if (stages.size > retainedStages) { - val toRemove = calculateNumberToRemove(stages.size, retainedStages) - stages.take(toRemove).foreach { s => - stageIdToData.remove((s.stageId, s.attemptId)) - stageIdToInfo.remove(s.stageId) - } - stages.trimStart(toRemove) - } - } - - /** If jobs is too large, remove and garbage collect old jobs */ - private def trimJobsIfNecessary(jobs: ListBuffer[JobUIData]) = synchronized { - if (jobs.size > retainedJobs) { - val toRemove = calculateNumberToRemove(jobs.size, retainedJobs) - jobs.take(toRemove).foreach { job => - // Remove the job's UI data, if it exists - jobIdToData.remove(job.jobId).foreach { removedJob => - // A null jobGroupId is used for jobs that are run without a job group - val jobGroupId = removedJob.jobGroup.orNull - // Remove the job group -> job mapping entry, if it exists - jobGroupToJobIds.get(jobGroupId).foreach { jobsInGroup => - jobsInGroup.remove(job.jobId) - // If this was the last job in this job group, remove the map entry for the job group - if (jobsInGroup.isEmpty) { - jobGroupToJobIds.remove(jobGroupId) - } - } - } - } - jobs.trimStart(toRemove) - } - } - - override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { - val jobGroup = for ( - props <- Option(jobStart.properties); - group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) - ) yield group - val jobData: JobUIData = - new JobUIData( - jobId = jobStart.jobId, - submissionTime = Option(jobStart.time).filter(_ >= 0), - stageIds = jobStart.stageIds, - jobGroup = jobGroup, - status = JobExecutionStatus.RUNNING) - // A null jobGroupId is used for jobs that are run without a job group - jobGroupToJobIds.getOrElseUpdate(jobGroup.orNull, new HashSet[JobId]).add(jobStart.jobId) - jobStart.stageInfos.foreach(x => pendingStages(x.stageId) = x) - // Compute (a potential underestimate of) the number of tasks that will be run by this job. - // This may be an underestimate because the job start event references all of the result - // stages' transitive stage dependencies, but some of these stages might be skipped if their - // output is available from earlier runs. - // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. - jobData.numTasks = { - val allStages = jobStart.stageInfos - val missingStages = allStages.filter(_.completionTime.isEmpty) - missingStages.map(_.numTasks).sum - } - jobIdToData(jobStart.jobId) = jobData - activeJobs(jobStart.jobId) = jobData - for (stageId <- jobStart.stageIds) { - stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId) - } - // If there's no information for a stage, store the StageInfo received from the scheduler - // so that we can display stage descriptions for pending stages: - for (stageInfo <- jobStart.stageInfos) { - stageIdToInfo.getOrElseUpdate(stageInfo.stageId, stageInfo) - stageIdToData.getOrElseUpdate((stageInfo.stageId, stageInfo.attemptId), new StageUIData) - } - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized { - val jobData = activeJobs.remove(jobEnd.jobId).getOrElse { - logWarning(s"Job completed for unknown job ${jobEnd.jobId}") - new JobUIData(jobId = jobEnd.jobId) - } - jobData.completionTime = Option(jobEnd.time).filter(_ >= 0) - - jobData.stageIds.foreach(pendingStages.remove) - jobEnd.jobResult match { - case JobSucceeded => - completedJobs += jobData - trimJobsIfNecessary(completedJobs) - jobData.status = JobExecutionStatus.SUCCEEDED - numCompletedJobs += 1 - case JobFailed(_) => - failedJobs += jobData - trimJobsIfNecessary(failedJobs) - jobData.status = JobExecutionStatus.FAILED - numFailedJobs += 1 - } - for (stageId <- jobData.stageIds) { - stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => - jobsUsingStage.remove(jobEnd.jobId) - if (jobsUsingStage.isEmpty) { - stageIdToActiveJobIds.remove(stageId) - } - stageIdToInfo.get(stageId).foreach { stageInfo => - if (stageInfo.submissionTime.isEmpty) { - // if this stage is pending, it won't complete, so mark it as "skipped": - skippedStages += stageInfo - trimStagesIfNecessary(skippedStages) - jobData.numSkippedStages += 1 - jobData.numSkippedTasks += stageInfo.numTasks - } - } - } - } - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { - val stage = stageCompleted.stageInfo - stageIdToInfo(stage.stageId) = stage - val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { - logWarning("Stage completed for unknown stage " + stage.stageId) - new StageUIData - }) - - for ((id, info) <- stageCompleted.stageInfo.accumulables) { - stageData.accumulables(id) = info - } - - poolToActiveStages.get(stageData.schedulingPool).foreach { hashMap => - hashMap.remove(stage.stageId) - } - activeStages.remove(stage.stageId) - if (stage.failureReason.isEmpty) { - completedStages += stage - numCompletedStages += 1 - trimStagesIfNecessary(completedStages) - } else { - failedStages += stage - numFailedStages += 1 - trimStagesIfNecessary(failedStages) - } - - for ( - activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); - jobId <- activeJobsDependentOnStage; - jobData <- jobIdToData.get(jobId) - ) { - jobData.numActiveStages -= 1 - if (stage.failureReason.isEmpty) { - if (stage.submissionTime.isDefined) { - jobData.completedStageIndices.add(stage.stageId) - } - } else { - jobData.numFailedStages += 1 - } - } - } - - /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { - val stage = stageSubmitted.stageInfo - activeStages(stage.stageId) = stage - pendingStages.remove(stage.stageId) - val poolName = Option(stageSubmitted.properties).map { - p => p.getProperty("spark.scheduler.pool", SparkUI.DEFAULT_POOL_NAME) - }.getOrElse(SparkUI.DEFAULT_POOL_NAME) - - stageIdToInfo(stage.stageId) = stage - val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) - stageData.schedulingPool = poolName - - stageData.description = Option(stageSubmitted.properties).flatMap { - p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) - } - - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) - stages(stage.stageId) = stage - - for ( - activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); - jobId <- activeJobsDependentOnStage; - jobData <- jobIdToData.get(jobId) - ) { - jobData.numActiveStages += 1 - - // If a stage retries again, it should be removed from completedStageIndices set - jobData.completedStageIndices.remove(stage.stageId) - } - } - - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { - val taskInfo = taskStart.taskInfo - if (taskInfo != null) { - val metrics = TaskMetrics.empty - val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { - logWarning("Task start for unknown stage " + taskStart.stageId) - new StageUIData - }) - stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, TaskUIData(taskInfo, Some(metrics))) - } - for ( - activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); - jobId <- activeJobsDependentOnStage; - jobData <- jobIdToData.get(jobId) - ) { - jobData.numActiveTasks += 1 - } - } - - 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. - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val info = taskEnd.taskInfo - // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task - // completion event is for. Let's just drop it here. This means we might have some speculation - // tasks on the web ui that's never marked as complete. - if (info != null && taskEnd.stageAttemptId != -1) { - val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), { - logWarning("Task end for unknown stage " + taskEnd.stageId) - new StageUIData - }) - - for (accumulableInfo <- info.accumulables) { - stageData.accumulables(accumulableInfo.id) = accumulableInfo - } - - val execSummaryMap = stageData.executorSummary - val execSummary = execSummaryMap.getOrElseUpdate(info.executorId, new ExecutorSummary) - - taskEnd.reason match { - case Success => - execSummary.succeededTasks += 1 - case kill: TaskKilled => - execSummary.reasonToNumKilled = execSummary.reasonToNumKilled.updated( - kill.reason, execSummary.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) - case _ => - execSummary.failedTasks += 1 - } - execSummary.taskTime += info.duration - stageData.numActiveTasks -= 1 - - val errorMessage: Option[String] = - taskEnd.reason match { - case org.apache.spark.Success => - stageData.completedIndices.add(info.index) - stageData.numCompleteTasks += 1 - None - case kill: TaskKilled => - stageData.reasonToNumKilled = stageData.reasonToNumKilled.updated( - kill.reason, stageData.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) - Some(kill.toErrorString) - case e: ExceptionFailure => // Handle ExceptionFailure because we might have accumUpdates - stageData.numFailedTasks += 1 - Some(e.toErrorString) - case e: TaskFailedReason => // All other failure cases - stageData.numFailedTasks += 1 - Some(e.toErrorString) - } - - val taskMetrics = Option(taskEnd.taskMetrics) - taskMetrics.foreach { m => - val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) - updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) - } - - val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) - taskData.updateTaskInfo(info) - taskData.updateTaskMetrics(taskMetrics) - taskData.errorMessage = errorMessage - - // If Tasks is too large, remove and garbage collect old tasks - if (stageData.taskData.size > retainedTasks) { - stageData.taskData = stageData.taskData.drop( - calculateNumberToRemove(stageData.taskData.size, retainedTasks)) - } - - for ( - activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); - jobId <- activeJobsDependentOnStage; - jobData <- jobIdToData.get(jobId) - ) { - jobData.numActiveTasks -= 1 - taskEnd.reason match { - case Success => - jobData.numCompletedTasks += 1 - case kill: TaskKilled => - jobData.reasonToNumKilled = jobData.reasonToNumKilled.updated( - kill.reason, jobData.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) - case _ => - jobData.numFailedTasks += 1 - } - } - } - } - - /** - * Remove at least (maxRetained / 10) items to reduce friction. - */ - private def calculateNumberToRemove(dataSize: Int, retainedSize: Int): Int = { - math.max(retainedSize / 10, dataSize - retainedSize) - } - - /** - * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage - * aggregate metrics by calculating deltas between the currently recorded metrics and the new - * metrics. - */ - def updateAggregateMetrics( - stageData: StageUIData, - execId: String, - taskMetrics: TaskMetrics, - oldMetrics: Option[TaskMetricsUIData]) { - val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) - - val shuffleWriteDelta = - taskMetrics.shuffleWriteMetrics.bytesWritten - - oldMetrics.map(_.shuffleWriteMetrics.bytesWritten).getOrElse(0L) - stageData.shuffleWriteBytes += shuffleWriteDelta - execSummary.shuffleWrite += shuffleWriteDelta - - val shuffleWriteRecordsDelta = - taskMetrics.shuffleWriteMetrics.recordsWritten - - oldMetrics.map(_.shuffleWriteMetrics.recordsWritten).getOrElse(0L) - stageData.shuffleWriteRecords += shuffleWriteRecordsDelta - execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta - - val shuffleReadDelta = - taskMetrics.shuffleReadMetrics.totalBytesRead - - oldMetrics.map(_.shuffleReadMetrics.totalBytesRead).getOrElse(0L) - stageData.shuffleReadTotalBytes += shuffleReadDelta - execSummary.shuffleRead += shuffleReadDelta - - val shuffleReadRecordsDelta = - taskMetrics.shuffleReadMetrics.recordsRead - - oldMetrics.map(_.shuffleReadMetrics.recordsRead).getOrElse(0L) - stageData.shuffleReadRecords += shuffleReadRecordsDelta - execSummary.shuffleReadRecords += shuffleReadRecordsDelta - - val inputBytesDelta = - taskMetrics.inputMetrics.bytesRead - - oldMetrics.map(_.inputMetrics.bytesRead).getOrElse(0L) - stageData.inputBytes += inputBytesDelta - execSummary.inputBytes += inputBytesDelta - - val inputRecordsDelta = - taskMetrics.inputMetrics.recordsRead - - oldMetrics.map(_.inputMetrics.recordsRead).getOrElse(0L) - stageData.inputRecords += inputRecordsDelta - execSummary.inputRecords += inputRecordsDelta - - val outputBytesDelta = - taskMetrics.outputMetrics.bytesWritten - - oldMetrics.map(_.outputMetrics.bytesWritten).getOrElse(0L) - stageData.outputBytes += outputBytesDelta - execSummary.outputBytes += outputBytesDelta - - val outputRecordsDelta = - taskMetrics.outputMetrics.recordsWritten - - oldMetrics.map(_.outputMetrics.recordsWritten).getOrElse(0L) - stageData.outputRecords += outputRecordsDelta - execSummary.outputRecords += outputRecordsDelta - - val diskSpillDelta = - taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) - stageData.diskBytesSpilled += diskSpillDelta - execSummary.diskBytesSpilled += diskSpillDelta - - val memorySpillDelta = - taskMetrics.memoryBytesSpilled - oldMetrics.map(_.memoryBytesSpilled).getOrElse(0L) - stageData.memoryBytesSpilled += memorySpillDelta - execSummary.memoryBytesSpilled += memorySpillDelta - - val timeDelta = - taskMetrics.executorRunTime - oldMetrics.map(_.executorRunTime).getOrElse(0L) - stageData.executorRunTime += timeDelta - - val cpuTimeDelta = - taskMetrics.executorCpuTime - oldMetrics.map(_.executorCpuTime).getOrElse(0L) - stageData.executorCpuTime += cpuTimeDelta - } - - override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - for ((taskId, sid, sAttempt, accumUpdates) <- executorMetricsUpdate.accumUpdates) { - val stageData = stageIdToData.getOrElseUpdate((sid, sAttempt), { - logWarning("Metrics update for task in unknown stage " + sid) - new StageUIData - }) - val taskData = stageData.taskData.get(taskId) - val metrics = TaskMetrics.fromAccumulatorInfos(accumUpdates) - taskData.foreach { t => - if (!t.taskInfo.finished) { - updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) - // Overwrite task metrics - t.updateTaskMetrics(Some(metrics)) - } - } - } - } - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - schedulingMode = environmentUpdate - .environmentDetails("Spark Properties").toMap - .get("spark.scheduler.mode") - .map(SchedulingMode.withName) - } - } - - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { - synchronized { - val blockManagerId = blockManagerAdded.blockManagerId - val executorId = blockManagerId.executorId - executorIdToBlockManagerId(executorId) = blockManagerId - } - } - - override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { - synchronized { - val executorId = blockManagerRemoved.blockManagerId.executorId - executorIdToBlockManagerId.remove(executorId) - } - } - - override def onApplicationStart(appStarted: SparkListenerApplicationStart) { - startTime = appStarted.time - } - - override def onApplicationEnd(appEnded: SparkListenerApplicationEnd) { - endTime = appEnded.time - } - - /** - * For testing only. Wait until at least `numExecutors` executors are up, or throw - * `TimeoutException` if the waiting time elapsed before `numExecutors` executors up. - * Exposed for testing. - * - * @param numExecutors the number of executors to wait at least - * @param timeout time to wait in milliseconds - */ - private[spark] def waitUntilExecutorsUp(numExecutors: Int, timeout: Long): Unit = { - val finishTime = System.currentTimeMillis() + timeout - while (System.currentTimeMillis() < finishTime) { - val numBlockManagers = synchronized { - blockManagerIds.size - } - if (numBlockManagers >= numExecutors + 1) { - // Need to count the block manager in driver - return - } - // 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) - } - throw new TimeoutException( - s"Can't find $numExecutors executors before $timeout milliseconds elapsed") - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 620c54c2dc0a5..74ccc1a75417a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -19,35 +19,50 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest +import scala.collection.JavaConverters._ + +import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.status.AppStateStore +import org.apache.spark.ui._ /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { +private[ui] class JobsTab(parent: SparkUI, store: AppStateStore) + extends SparkUITab(parent, "jobs") { + val sc = parent.sc val killEnabled = parent.killEnabled - val jobProgresslistener = parent.jobProgressListener - val executorListener = parent.executorsListener - val operationGraphListener = parent.operationGraphListener - def isFairScheduler: Boolean = - jobProgresslistener.schedulingMode == Some(SchedulingMode.FAIR) + def isFairScheduler: Boolean = { + val configName = "spark.scheduler.mode" + val config = sc match { + case Some(_sc) => + _sc.conf.getOption(configName) + case _ => + store.environmentInfo().sparkProperties.toMap.get(configName) + } + config.map(SchedulingMode.withName) == Some(SchedulingMode.FAIR) + } def getSparkUser: String = parent.getSparkUser - attachPage(new AllJobsPage(this)) - attachPage(new JobPage(this)) + attachPage(new AllJobsPage(this, store)) + attachPage(new JobPage(this, store)) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - val jobId = Option(request.getParameter("id")).map(_.toInt) - jobId.foreach { id => - if (jobProgresslistener.activeJobs.contains(id)) { - sc.foreach(_.cancelJob(id)) - // Do a quick pause here to give Spark time to kill the job so it shows up as - // killed after the refresh. Note that this will block the serving thread so the - // time should be limited in duration. - Thread.sleep(100) + Option(request.getParameter("id")).map(_.toInt).foreach { id => + try { + val job = store.job(id) + if (job.status == JobExecutionStatus.RUNNING) { + sc.foreach(_.cancelJob(id)) + // Do a quick pause here to give Spark time to kill the job so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } catch { + case _: NoSuchElementException => } } } 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 8ee70d27cc09f..fcb56569f3cc9 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 @@ -17,49 +17,50 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1._ import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing specific pool details */ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { - private val sc = parent.sc - private val listener = parent.progressListener def render(request: HttpServletRequest): Seq[Node] = { - listener.synchronized { - val poolName = Option(request.getParameter("poolname")).map { poolname => - UIUtils.decodeURLParameter(poolname) - }.getOrElse { - throw new IllegalArgumentException(s"Missing poolname parameter") - } + val poolName = Option(request.getParameter("poolname")).map { poolname => + UIUtils.decodeURLParameter(poolname) + }.getOrElse { + throw new IllegalArgumentException(s"Missing poolname parameter") + } - val poolToActiveStages = listener.poolToActiveStages - val activeStages = poolToActiveStages.get(poolName) match { - case Some(s) => s.values.toSeq - case None => Seq[StageInfo]() - } - val shouldShowActiveStages = activeStages.nonEmpty - val activeStagesTable = - new StageTableBase(request, activeStages, "", "activeStage", parent.basePath, "stages/pool", - parent.progressListener, parent.isFairScheduler, parent.killEnabled, - isFailedStage = false) + // For now, pool information is only accessible in live UIs + val pool = parent.sc.flatMap(_.getPoolForName(poolName)).getOrElse { + throw new IllegalArgumentException(s"Unknown pool: $poolName") + } - // For now, pool information is only accessible in live UIs - val pools = sc.map(_.getPoolForName(poolName).getOrElse { - throw new IllegalArgumentException(s"Unknown poolname: $poolName") - }).toSeq - val poolTable = new PoolTable(pools, parent) + val uiPool = try { + parent.store.pool(poolName) + } catch { + case _: NoSuchElementException => + new PoolData(poolName, Set()) + } - var content =

Summary

++ poolTable.toNodeSeq - if (shouldShowActiveStages) { - content ++=

{activeStages.size} Active Stages

++ activeStagesTable.toNodeSeq - } + val activeStages = uiPool.stageIds.toSeq.map(parent.store.lastStageAttempt(_)) - UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) + val shouldShowActiveStages = activeStages.nonEmpty + val activeStagesTable = + new StageTableBase(parent.store, request, activeStages, "", "activeStage", parent.basePath, + "stages/pool", parent.isFairScheduler, parent.killEnabled, false) + + val poolTable = new PoolTable(Map(pool -> uiPool), parent) + var content =

Summary

++ poolTable.toNodeSeq + if (shouldShowActiveStages) { + content ++=

{activeStages.size} Active Stages

++ activeStagesTable.toNodeSeq } + + UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) } } 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 ea02968733cac..ddc42eea26704 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 @@ -22,22 +22,15 @@ import java.net.URLEncoder import scala.collection.mutable.HashMap import scala.xml.Node -import org.apache.spark.scheduler.{Schedulable, StageInfo} +import org.apache.spark.scheduler.Schedulable +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1.StageData import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { - private val listener = parent.progressListener +private[ui] class PoolTable(pools: Map[Schedulable, PoolData], parent: StagesTab) { def toNodeSeq: Seq[Node] = { - listener.synchronized { - poolTable(poolRow, pools) - } - } - - private def poolTable( - makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], - rows: Seq[Schedulable]): Seq[Node] = { @@ -48,29 +41,24 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { - {rows.map(r => makeRow(r, listener.poolToActiveStages))} + {pools.map { case (s, p) => poolRow(s, p) }}
Pool NameSchedulingMode
} - 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 - } + private def poolRow(s: Schedulable, p: PoolData): Seq[Node] = { + val activeStages = p.stageIds.size val href = "%s/stages/pool?poolname=%s" .format(UIUtils.prependBaseUri(parent.basePath), URLEncoder.encode(p.name, "UTF-8")) {p.name} - {p.minShare} - {p.weight} + {s.minShare} + {s.weight} {activeStages} - {p.runningTasks} - {p.schedulingMode} + {s.runningTasks} + {s.schedulingMode} } } 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 19325a2dc9169..87974ab7c81ea 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 @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs import java.net.URLEncoder -import java.util.Date +import java.util.{Date, NoSuchElementException} import javax.servlet.http.HttpServletRequest import scala.collection.mutable.HashSet @@ -27,21 +27,18 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.SparkConf -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.{TaskInfo, TaskLocality} +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1._ import org.apache.spark.ui._ -import org.apache.spark.ui.exec.ExecutorsListener -import org.apache.spark.ui.jobs.UIData._ +import org.apache.spark.ui.jobs.ApiHelper._ import org.apache.spark.util.{Distribution, Utils} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +private[ui] class StagePage(parent: StagesTab, store: AppStateStore) extends WebUIPage("stage") { import StagePage._ - private val progressListener = parent.progressListener - private val operationGraphListener = parent.operationGraphListener - private val executorsListener = parent.executorsListener - private val TIMELINE_LEGEND = {
@@ -70,553 +67,529 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // if we find that it's okay. private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) - private def getLocalitySummaryString(stageData: StageUIData): String = { - val localities = stageData.taskData.values.map(_.taskInfo.taskLocality) + private def getLocalitySummaryString(stageData: StageData, taskList: Seq[TaskData]): String = { + val localities = taskList.map(_.taskLocality) val localityCounts = localities.groupBy(identity).mapValues(_.size) + val names = Map( + TaskLocality.PROCESS_LOCAL.toString() -> "Process local", + TaskLocality.NODE_LOCAL.toString() -> "Node local", + TaskLocality.RACK_LOCAL.toString() -> "Rack local", + TaskLocality.ANY.toString() -> "Any") val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) => - val localityName = locality match { - case TaskLocality.PROCESS_LOCAL => "Process local" - case TaskLocality.NODE_LOCAL => "Node local" - case TaskLocality.RACK_LOCAL => "Rack local" - case TaskLocality.ANY => "Any" - } - s"$localityName: $count" + s"${names(locality)}: $count" } localityNamesAndCounts.sorted.mkString("; ") } def render(request: HttpServletRequest): Seq[Node] = { - progressListener.synchronized { - val parameterId = request.getParameter("id") - require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - - val parameterAttempt = request.getParameter("attempt") - require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - - val parameterTaskPage = request.getParameter("task.page") - val parameterTaskSortColumn = request.getParameter("task.sort") - val parameterTaskSortDesc = request.getParameter("task.desc") - val parameterTaskPageSize = request.getParameter("task.pageSize") - val parameterTaskPrevPageSize = request.getParameter("task.prevPageSize") - - val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) - val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => - UIUtils.decodeURLParameter(sortColumn) - }.getOrElse("Index") - val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) - val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) - val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) - - val stageId = parameterId.toInt - val stageAttemptId = parameterAttempt.toInt - val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) - - val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - if (stageDataOption.isEmpty) { + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val parameterAttempt = request.getParameter("attempt") + require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") + + val parameterTaskPage = request.getParameter("task.page") + val parameterTaskSortColumn = request.getParameter("task.sort") + val parameterTaskSortDesc = request.getParameter("task.desc") + val parameterTaskPageSize = request.getParameter("task.pageSize") + val parameterTaskPrevPageSize = request.getParameter("task.prevPageSize") + + val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) + val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Index") + val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) + val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) + val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) + + val stageId = parameterId.toInt + val stageAttemptId = parameterAttempt.toInt + + val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" + val stageData = try { + parent.store.stageAttempt(stageId, stageAttemptId) + } catch { + case _: NoSuchElementException => val content =

No information to display for Stage {stageId} (Attempt {stageAttemptId})

return UIUtils.headerSparkPage(stageHeader, content, parent) + } - } - if (stageDataOption.get.taskData.isEmpty) { - val content = -
-

Summary Metrics

No tasks have started yet -

Tasks

No tasks have started yet -
- return UIUtils.headerSparkPage(stageHeader, content, parent) - } + // Retrieve at most the configured max tasks from the store, most recent first. + val maxTasks = parent.conf.get(UI_RETAINED_TASKS) + val taskList = parent.store.taskList(stageId, stageAttemptId, maxTasks) - val stageData = stageDataOption.get - val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = stageData.numCompleteTasks - val totalTasks = stageData.numActiveTasks + - stageData.numCompleteTasks + stageData.numFailedTasks - val totalTasksNumStr = if (totalTasks == tasks.size) { - s"$totalTasks" - } else { - s"$totalTasks, showing ${tasks.size}" - } + if (taskList.isEmpty) { + val content = +
+

Summary Metrics

No tasks have started yet +

Tasks

No tasks have started yet +
+ return UIUtils.headerSparkPage(stageHeader, content, parent) + } - val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables - val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal } - val hasAccumulators = externalAccumulables.nonEmpty + val tasks = taskList.sortBy(_.launchTime) + val numCompleted = stageData.numCompleteTasks + val totalTasks = stageData.numActiveTasks + + stageData.numCompleteTasks + stageData.numFailedTasks + val totalTasksNumStr = if (totalTasks == tasks.size) { + s"$totalTasks" + } else { + s"$totalTasks, showing ${tasks.size}" + } - val summary = -
-
    + val externalAccumulables = stageData.accumulatorUpdates + val hasAccumulators = externalAccumulables.size > 0 + + val summary = +
    +
      +
    • + Total Time Across All Tasks: + {UIUtils.formatDuration(stageData.executorRunTime)} +
    • +
    • + Locality Level Summary: + {getLocalitySummaryString(stageData, taskList)} +
    • + {if (hasInput(stageData)) {
    • - Total Time Across All Tasks: - {UIUtils.formatDuration(stageData.executorRunTime)} + Input Size / Records: + {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"}
    • + }} + {if (hasOutput(stageData)) {
    • - Locality Level Summary: - {getLocalitySummaryString(stageData)} + Output: + {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"}
    • - {if (stageData.hasInput) { -
    • - Input Size / Records: - {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"} -
    • - }} - {if (stageData.hasOutput) { -
    • - Output: - {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"} -
    • - }} - {if (stageData.hasShuffleRead) { -
    • - Shuffle Read: - {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + - s"${stageData.shuffleReadRecords}"} -
    • - }} - {if (stageData.hasShuffleWrite) { -
    • - Shuffle Write: - {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + - s"${stageData.shuffleWriteRecords}"} -
    • - }} - {if (stageData.hasBytesSpilled) { -
    • - Shuffle Spill (Memory): - {Utils.bytesToString(stageData.memoryBytesSpilled)} -
    • -
    • - Shuffle Spill (Disk): - {Utils.bytesToString(stageData.diskBytesSpilled)} -
    • - }} -
    -
    + }} + {if (hasShuffleRead(stageData)) { +
  • + Shuffle Read: + {s"${Utils.bytesToString(stageData.shuffleReadBytes)} / " + + s"${stageData.shuffleReadRecords}"} +
  • + }} + {if (hasShuffleWrite(stageData)) { +
  • + Shuffle Write: + {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + + s"${stageData.shuffleWriteRecords}"} +
  • + }} + {if (hasBytesSpilled(stageData)) { +
  • + Shuffle Spill (Memory): + {Utils.bytesToString(stageData.memoryBytesSpilled)} +
  • +
  • + Shuffle Spill (Disk): + {Utils.bytesToString(stageData.diskBytesSpilled)} +
  • + }} +
+
- val showAdditionalMetrics = -
- - - Show Additional Metrics - - +
- val outputSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.bytesWritten.toDouble - } + val stageGraph = try { + Some(parent.store.operationGraphForStage(stageId)) + } catch { + case _: NoSuchElementException => None + } + val dagViz = UIUtils.showDagVizForStage(stageId, stageGraph) - val outputRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.recordsWritten.toDouble - } + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") + def accumulableRow(acc: AccumulableInfo): Seq[Node] = { + {acc.name}{acc.value} + } + val accumulableTable = UIUtils.listingTable( + accumulableHeaders, + accumulableRow, + externalAccumulables.toSeq) + + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (taskPageSize <= taskPrevPageSize) { + taskPage + } else { + 1 + } + } + val currentTime = System.currentTimeMillis() + val (taskTable, taskTableHTML) = try { + val _taskTable = new TaskPagedTable( + parent.conf, + UIUtils.prependBaseUri(parent.basePath) + + s"/stages/stage?id=${stageId}&attempt=${stageAttemptId}", + tasks, + hasAccumulators, + hasInput(stageData), + hasOutput(stageData), + hasShuffleRead(stageData), + hasShuffleWrite(stageData), + hasBytesSpilled(stageData), + currentTime, + pageSize = taskPageSize, + sortColumn = taskSortColumn, + desc = taskSortDesc, + store = parent.store + ) + (_taskTable, _taskTable.table(page)) + } catch { + case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => + val errorMessage = +
+

Error while rendering stage table:

+
+              {Utils.exceptionString(e)}
+            
+
+ (null, errorMessage) + } - val outputQuantiles = Output Size / Records +: - getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) + val jsForScrollingDownToTaskTable = + - val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.fetchWaitTime.toDouble - } - val shuffleReadBlockedQuantiles = - - - Shuffle Read Blocked Time - - +: - getFormattedTimeQuantiles(shuffleReadBlockedTimes) + val taskIdsInPage = if (taskTable == null) Set.empty[Long] + else taskTable.dataSource.slicedTaskIds - val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.totalBytesRead.toDouble - } - val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.recordsRead.toDouble - } - val shuffleReadTotalQuantiles = - - - Shuffle Read Size / Records - - +: - getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) + // Excludes tasks which failed and have incomplete metrics + val validTasks = tasks.filter(t => t.status == "SUCCESS" && t.taskMetrics.isDefined) - val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.remoteBytesRead.toDouble + val summaryTable: Option[Seq[Node]] = + if (validTasks.size == 0) { + None + } + else { + def getDistributionQuantiles(data: Seq[Double]): IndexedSeq[Double] = + Distribution(data).get.getQuantiles() + def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { + getDistributionQuantiles(times).map { millis => + {UIUtils.formatDuration(millis.toLong)} } - val shuffleReadRemoteQuantiles = - - - Shuffle Remote Reads - - +: - getFormattedSizeQuantiles(shuffleReadRemoteSizes) + } + def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = { + getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) + } - val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.bytesWritten.toDouble - } + val deserializationTimes = validTasks.map { task => + task.taskMetrics.get.executorDeserializeTime.toDouble + } + val deserializationQuantiles = + + + Task Deserialization Time + + +: getFormattedTimeQuantiles(deserializationTimes) + + val serviceTimes = validTasks.map(_.taskMetrics.get.executorRunTime.toDouble) + val serviceQuantiles = Duration +: getFormattedTimeQuantiles(serviceTimes) + + val gcTimes = validTasks.map(_.taskMetrics.get.jvmGcTime.toDouble) + val gcQuantiles = + + GC Time + + +: getFormattedTimeQuantiles(gcTimes) + + val serializationTimes = validTasks.map(_.taskMetrics.get.resultSerializationTime.toDouble) + val serializationQuantiles = + + + Result Serialization Time + + +: getFormattedTimeQuantiles(serializationTimes) + + val gettingResultTimes = validTasks.map(getGettingResultTime(_, currentTime).toDouble) + val gettingResultQuantiles = + + + Getting Result Time + + +: + getFormattedTimeQuantiles(gettingResultTimes) + + val peakExecutionMemory = validTasks.map(_.taskMetrics.get.peakExecutionMemory.toDouble) + val peakExecutionMemoryQuantiles = { + + + Peak Execution Memory + + +: getFormattedSizeQuantiles(peakExecutionMemory) + } - val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.recordsWritten.toDouble - } + // 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 { task => + getSchedulerDelay(task, task.taskMetrics.get, currentTime).toDouble + } + val schedulerDelayTitle = Scheduler Delay + val schedulerDelayQuantiles = schedulerDelayTitle +: + getFormattedTimeQuantiles(schedulerDelays) + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) + : Seq[Elem] = { + val recordDist = getDistributionQuantiles(records).iterator + getDistributionQuantiles(data).map(d => + {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} + ) + } - val shuffleWriteQuantiles = Shuffle Write Size / Records +: - getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) + val inputSizes = validTasks.map(_.taskMetrics.get.inputMetrics.bytesRead.toDouble) + val inputRecords = validTasks.map(_.taskMetrics.get.inputMetrics.recordsRead.toDouble) + val inputQuantiles = Input Size / Records +: + getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) - val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.memoryBytesSpilled.toDouble - } - val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: - getFormattedSizeQuantiles(memoryBytesSpilledSizes) + val outputSizes = validTasks.map(_.taskMetrics.get.outputMetrics.bytesWritten.toDouble) + val outputRecords = validTasks.map(_.taskMetrics.get.outputMetrics.recordsWritten.toDouble) + val outputQuantiles = Output Size / Records +: + getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) - val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.diskBytesSpilled.toDouble - } - val diskBytesSpilledQuantiles = Shuffle spill (disk) +: - getFormattedSizeQuantiles(diskBytesSpilledSizes) - - val listings: Seq[Seq[Node]] = Seq( - {serviceQuantiles}, - {schedulerDelayQuantiles}, - - {deserializationQuantiles} - - {gcQuantiles}, - - {serializationQuantiles} - , - {gettingResultQuantiles}, - - {peakExecutionMemoryQuantiles} - , - if (stageData.hasInput) {inputQuantiles} else Nil, - if (stageData.hasOutput) {outputQuantiles} else Nil, - if (stageData.hasShuffleRead) { - - {shuffleReadBlockedQuantiles} - - {shuffleReadTotalQuantiles} - - {shuffleReadRemoteQuantiles} - - } else { - Nil - }, - if (stageData.hasShuffleWrite) {shuffleWriteQuantiles} else Nil, - if (stageData.hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, - if (stageData.hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) - - val quantileHeaders = Seq("Metric", "Min", "25th percentile", - "Median", "75th percentile", "Max") - // The summary table does not use CSS to stripe rows, which doesn't work with hidden - // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). - Some(UIUtils.listingTable( - quantileHeaders, - identity[Seq[Node]], - listings, - fixedWidth = true, - id = Some("task-summary-table"), - stripeRowsWithCss = false)) + val shuffleReadBlockedTimes = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.fetchWaitTime.toDouble + } + val shuffleReadBlockedQuantiles = + + + Shuffle Read Blocked Time + + +: + getFormattedTimeQuantiles(shuffleReadBlockedTimes) + + val shuffleReadTotalSizes = validTasks.map { task => + totalBytesRead(task.taskMetrics.get.shuffleReadMetrics).toDouble + } + val shuffleReadTotalRecords = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.recordsRead.toDouble + } + val shuffleReadTotalQuantiles = + + + Shuffle Read Size / Records + + +: + getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) + + val shuffleReadRemoteSizes = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.remoteBytesRead.toDouble + } + val shuffleReadRemoteQuantiles = + + + Shuffle Remote Reads + + +: + getFormattedSizeQuantiles(shuffleReadRemoteSizes) + + val shuffleWriteSizes = validTasks.map { task => + task.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toDouble } - val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) + val shuffleWriteRecords = validTasks.map { task => + task.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toDouble + } - val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators) {

Accumulators

++ accumulableTable } else Seq() + val shuffleWriteQuantiles = Shuffle Write Size / Records +: + getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) + + val memoryBytesSpilledSizes = validTasks.map(_.taskMetrics.get.memoryBytesSpilled.toDouble) + val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: + getFormattedSizeQuantiles(memoryBytesSpilledSizes) + + val diskBytesSpilledSizes = validTasks.map(_.taskMetrics.get.diskBytesSpilled.toDouble) + val diskBytesSpilledQuantiles = Shuffle spill (disk) +: + getFormattedSizeQuantiles(diskBytesSpilledSizes) + + val listings: Seq[Seq[Node]] = Seq( + {serviceQuantiles}, + {schedulerDelayQuantiles}, + + {deserializationQuantiles} + + {gcQuantiles}, + + {serializationQuantiles} + , + {gettingResultQuantiles}, + + {peakExecutionMemoryQuantiles} + , + if (hasInput(stageData)) {inputQuantiles} else Nil, + if (hasOutput(stageData)) {outputQuantiles} else Nil, + if (hasShuffleRead(stageData)) { + + {shuffleReadBlockedQuantiles} + + {shuffleReadTotalQuantiles} + + {shuffleReadRemoteQuantiles} + + } else { + Nil + }, + if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil, + if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil, + if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25th percentile", + "Median", "75th percentile", "Max") + // The summary table does not use CSS to stripe rows, which doesn't work with hidden + // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). + Some(UIUtils.listingTable( + quantileHeaders, + identity[Seq[Node]], + listings, + fixedWidth = true, + id = Some("task-summary-table"), + stripeRowsWithCss = false)) + } - val aggMetrics = - -

- - Aggregated Metrics by Executor -

-
-
- {executorTable.toNodeSeq} -
+ val executorTable = new ExecutorTable(stageId, stageAttemptId, parent.store) + + val maybeAccumulableTable: Seq[Node] = + if (hasAccumulators) {

Accumulators

++ accumulableTable } else Seq() + + val aggMetrics = + +

+ + Aggregated Metrics by Executor +

+
+
+ {executorTable.toNodeSeq} +
- val content = - summary ++ - dagViz ++ - showAdditionalMetrics ++ - makeTimeline( - // Only show the tasks in the table - stageData.taskData.values.toSeq.filter(t => taskIdsInPage.contains(t.taskInfo.taskId)), - currentTime) ++ -

Summary Metrics for {numCompleted} Completed Tasks

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

Tasks ({totalTasksNumStr})

++ - taskTableHTML ++ jsForScrollingDownToTaskTable - UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) - } + val content = + summary ++ + dagViz ++ + showAdditionalMetrics ++ + makeTimeline( + // Only show the tasks in the table + taskList.filter { t => taskIdsInPage.contains(t.taskId) }, + currentTime) ++ +

Summary Metrics for {numCompleted} Completed Tasks

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

Tasks ({totalTasksNumStr})

++ + taskTableHTML ++ jsForScrollingDownToTaskTable + UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } - def makeTimeline(tasks: Seq[TaskUIData], currentTime: Long): Seq[Node] = { + def makeTimeline(tasks: Seq[TaskData], currentTime: Long): Seq[Node] = { val executorsSet = new HashSet[(String, String)] var minLaunchTime = Long.MaxValue var maxFinishTime = Long.MinValue val executorsArrayStr = - tasks.sortBy(-_.taskInfo.launchTime).take(MAX_TIMELINE_TASKS).map { taskUIData => - val taskInfo = taskUIData.taskInfo + tasks.sortBy(-_.launchTime.getTime()).take(MAX_TIMELINE_TASKS).map { taskInfo => val executorId = taskInfo.executorId val host = taskInfo.host executorsSet += ((executorId, host)) - val launchTime = taskInfo.launchTime - val finishTime = if (!taskInfo.running) taskInfo.finishTime else currentTime + val launchTime = taskInfo.launchTime.getTime() + val finishTime = taskInfo.duration.map(taskInfo.launchTime.getTime() + _) + .getOrElse(currentTime) val totalExecutionTime = finishTime - launchTime minLaunchTime = launchTime.min(minLaunchTime) maxFinishTime = finishTime.max(maxFinishTime) def toProportion(time: Long) = time.toDouble / totalExecutionTime * 100 - val metricsOpt = taskUIData.metrics + val metricsOpt = taskInfo.taskMetrics val shuffleReadTime = metricsOpt.map(_.shuffleReadMetrics.fetchWaitTime).getOrElse(0L) val shuffleReadTimeProportion = toProportion(shuffleReadTime) @@ -628,14 +601,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val serializationTimeProportion = toProportion(serializationTime) val deserializationTime = metricsOpt.map(_.executorDeserializeTime).getOrElse(0L) val deserializationTimeProportion = toProportion(deserializationTime) - val gettingResultTime = getGettingResultTime(taskUIData.taskInfo, currentTime) + val gettingResultTime = getGettingResultTime(taskInfo, currentTime) val gettingResultTimeProportion = toProportion(gettingResultTime) val schedulerDelay = metricsOpt.map(getSchedulerDelay(taskInfo, _, currentTime)).getOrElse(0L) val schedulerDelayProportion = toProportion(schedulerDelay) val executorOverhead = serializationTime + deserializationTime - val executorRunTime = if (taskInfo.running) { + val executorRunTime = if (taskInfo.duration.isDefined) { totalExecutionTime - executorOverhead - gettingResultTime } else { metricsOpt.map(_.executorRunTime).getOrElse( @@ -662,7 +635,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { serializationTimeProportionPos + serializationTimeProportion val index = taskInfo.index - val attempt = taskInfo.attemptNumber + val attempt = taskInfo.attempt val svgTag = if (totalExecutionTime == 0) { @@ -704,7 +677,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { |Status: ${taskInfo.status}
|Launch Time: ${UIUtils.formatDate(new Date(launchTime))} |${ - if (!taskInfo.running) { + if (!taskInfo.duration.isDefined) { s"""
Finish Time: ${UIUtils.formatDate(new Date(finishTime))}""" } else { "" @@ -769,34 +742,40 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } private[ui] object StagePage { - private[ui] def getGettingResultTime(info: TaskInfo, currentTime: Long): Long = { - if (info.gettingResult) { - if (info.finished) { - info.finishTime - info.gettingResultTime - } else { - // The task is still fetching the result. - currentTime - info.gettingResultTime - } - } else { - 0L + private[ui] def getGettingResultTime(info: TaskData, currentTime: Long): Long = { + info.resultFetchStart match { + case Some(start) => + info.duration match { + case Some(duration) => + info.launchTime.getTime() + duration - start.getTime() + + case _ => + currentTime - start.getTime() + } + + case _ => + 0L } } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { - if (info.finished) { - val totalExecutionTime = info.finishTime - info.launchTime - val executorOverhead = metrics.executorDeserializeTime + - metrics.resultSerializationTime - math.max( - 0, - totalExecutionTime - metrics.executorRunTime - executorOverhead - - getGettingResultTime(info, currentTime)) - } else { - // The task is still running and the metrics like executorRunTime are not available. - 0L + info: TaskData, + metrics: TaskMetrics, + currentTime: Long): Long = { + info.duration match { + case Some(duration) => + val executorOverhead = metrics.executorDeserializeTime + metrics.resultSerializationTime + math.max( + 0, + duration - metrics.executorRunTime - executorOverhead - + getGettingResultTime(info, currentTime)) + + case _ => + // The task is still running and the metrics like executorRunTime are not available. + 0L } } + } private[ui] case class TaskTableRowInputData(inputSortable: Long, inputReadable: String) @@ -825,7 +804,7 @@ private[ui] case class TaskTableRowBytesSpilledData( /** * Contains all data that needs for sorting and generating HTML. Using this one rather than - * TaskUIData to avoid creating duplicate contents during sorting the data. + * TaskData to avoid creating duplicate contents during sorting the data. */ private[ui] class TaskTableRowData( val index: Int, @@ -854,7 +833,7 @@ private[ui] class TaskTableRowData( val logs: Map[String, String]) private[ui] class TaskDataSource( - tasks: Seq[TaskUIData], + tasks: Seq[TaskData], hasAccumulators: Boolean, hasInput: Boolean, hasOutput: Boolean, @@ -865,10 +844,10 @@ private[ui] class TaskDataSource( pageSize: Int, sortColumn: String, desc: Boolean, - executorsListener: ExecutorsListener) extends PagedDataSource[TaskTableRowData](pageSize) { + store: AppStateStore) extends PagedDataSource[TaskTableRowData](pageSize) { import StagePage._ - // Convert TaskUIData to TaskTableRowData which contains the final contents to show in the table + // Convert TaskData to TaskTableRowData which contains the final contents to show in the table // so that we can avoid creating duplicate contents during sorting the data private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc)) @@ -884,25 +863,19 @@ private[ui] class TaskDataSource( def slicedTaskIds: Set[Long] = _slicedTaskIds - private def taskRow(taskData: TaskUIData): TaskTableRowData = { - val info = taskData.taskInfo - val metrics = taskData.metrics - val duration = taskData.taskDuration.getOrElse(1L) - val formatDuration = taskData.taskDuration.map(d => UIUtils.formatDuration(d)).getOrElse("") + private def taskRow(info: TaskData): TaskTableRowData = { + val metrics = info.taskMetrics + val duration = info.duration.getOrElse(1L) + val formatDuration = info.duration.map(d => UIUtils.formatDuration(d)).getOrElse("") val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L) - val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val gcTime = metrics.map(_.jvmGcTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val externalAccumulableReadable = info.accumulables - .filterNot(_.internal) - .flatMap { a => - (a.name, a.update) match { - case (Some(name), Some(update)) => Some(StringEscapeUtils.escapeHtml4(s"$name: $update")) - case _ => None - } - } + val externalAccumulableReadable = info.accumulatorUpdates.map { acc => + StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update}") + } val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L) val maybeInput = metrics.map(_.inputMetrics) @@ -924,7 +897,7 @@ private[ui] class TaskDataSource( val shuffleReadBlockedTimeReadable = maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") - val totalShuffleBytes = maybeShuffleRead.map(_.totalBytesRead) + val totalShuffleBytes = maybeShuffleRead.map(totalBytesRead) val shuffleReadSortable = totalShuffleBytes.getOrElse(0L) val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("") val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("") @@ -1007,17 +980,16 @@ private[ui] class TaskDataSource( None } - val logs = executorsListener.executorToTaskSummary.get(info.executorId) - .map(_.executorLogs).getOrElse(Map.empty) + val logs = store.executorSummary(info.executorId).map(_.executorLogs).getOrElse(Map.empty) new TaskTableRowData( info.index, info.taskId, - info.attemptNumber, + info.attempt, info.speculative, info.status, info.taskLocality.toString, s"${info.executorId} / ${info.host}", - info.launchTime, + info.launchTime.getTime(), duration, formatDuration, schedulerDelay, @@ -1032,7 +1004,7 @@ private[ui] class TaskDataSource( shuffleRead, shuffleWrite, bytesSpilled, - taskData.errorMessage.getOrElse(""), + info.errorMessage.getOrElse(""), logs) } @@ -1143,7 +1115,7 @@ private[ui] class TaskDataSource( private[ui] class TaskPagedTable( conf: SparkConf, basePath: String, - data: Seq[TaskUIData], + data: Seq[TaskData], hasAccumulators: Boolean, hasInput: Boolean, hasOutput: Boolean, @@ -1154,7 +1126,7 @@ private[ui] class TaskPagedTable( pageSize: Int, sortColumn: String, desc: Boolean, - executorsListener: ExecutorsListener) extends PagedTable[TaskTableRowData] { + store: AppStateStore) extends PagedTable[TaskTableRowData] { override def tableId: String = "task-table" @@ -1179,7 +1151,7 @@ private[ui] class TaskPagedTable( pageSize, sortColumn, desc, - executorsListener) + store) override def pageLink(page: Int): String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") @@ -1355,3 +1327,23 @@ private[ui] class TaskPagedTable( {errorSummary}{details} } } + +private object ApiHelper { + + def hasInput(stageData: StageData): Boolean = stageData.inputBytes > 0 + + def hasOutput(stageData: StageData): Boolean = stageData.outputBytes > 0 + + def hasShuffleRead(stageData: StageData): Boolean = stageData.shuffleReadBytes > 0 + + def hasShuffleWrite(stageData: StageData): Boolean = stageData.shuffleWriteBytes > 0 + + def hasBytesSpilled(stageData: StageData): Boolean = { + stageData.diskBytesSpilled > 0 || stageData.memoryBytesSpilled > 0 + } + + def totalBytesRead(metrics: ShuffleReadMetrics): Long = { + metrics.localBytesRead + metrics.remoteBytesRead + } + +} 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 256b726fa7eea..6b84bf980b193 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,18 +27,20 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1 +import org.apache.spark.storage.RDDInfo import org.apache.spark.ui._ -import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils private[ui] class StageTableBase( + store: AppStateStore, request: HttpServletRequest, - stages: Seq[StageInfo], + stages: Seq[v1.StageData], tableHeaderID: String, stageTag: String, basePath: String, subPath: String, - progressListener: JobProgressListener, isFairScheduler: Boolean, killEnabled: Boolean, isFailedStage: Boolean) { @@ -77,12 +79,12 @@ private[ui] class StageTableBase( val toNodeSeq = try { new StagePagedTable( + store, stages, tableHeaderID, stageTag, basePath, subPath, - progressListener, isFairScheduler, killEnabled, currentTime, @@ -104,13 +106,13 @@ private[ui] class StageTableBase( } private[ui] class StageTableRowData( - val stageInfo: StageInfo, - val stageData: Option[StageUIData], + val stage: v1.StageData, + val option: Option[v1.StageData], val stageId: Int, val attemptId: Int, val schedulingPool: String, val descriptionOption: Option[String], - val submissionTime: Long, + val submissionTime: Date, val formattedSubmissionTime: String, val duration: Long, val formattedDuration: String, @@ -124,19 +126,20 @@ private[ui] class StageTableRowData( val shuffleWriteWithUnit: String) private[ui] class MissingStageTableRowData( - stageInfo: StageInfo, + stageInfo: v1.StageData, stageId: Int, attemptId: Int) extends StageTableRowData( - stageInfo, None, stageId, attemptId, "", None, 0, "", -1, "", 0, "", 0, "", 0, "", 0, "") + stageInfo, None, stageId, attemptId, "", None, new Date(0), "", -1, "", 0, "", 0, "", 0, "", 0, + "") /** Page showing list of all ongoing and recently finished stages */ private[ui] class StagePagedTable( - stages: Seq[StageInfo], + store: AppStateStore, + stages: Seq[v1.StageData], tableHeaderId: String, stageTag: String, basePath: String, subPath: String, - listener: JobProgressListener, isFairScheduler: Boolean, killEnabled: Boolean, currentTime: Long, @@ -162,8 +165,8 @@ private[ui] class StagePagedTable( parameterOtherTable.mkString("&") override val dataSource = new StageDataSource( + store, stages, - listener, currentTime, pageSize, sortColumn, @@ -272,10 +275,10 @@ private[ui] class StagePagedTable( } private def rowContent(data: StageTableRowData): Seq[Node] = { - data.stageData match { + data.option match { case None => missingStageRow(data.stageId) case Some(stageData) => - val info = data.stageInfo + val info = data.stage {if (data.attemptId > 0) { {data.stageId} (retry {data.attemptId}) @@ -299,8 +302,8 @@ private[ui] class StagePagedTable( {data.formattedDuration} {UIUtils.makeProgressBar(started = stageData.numActiveTasks, - completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, - skipped = 0, reasonToNumKilled = stageData.reasonToNumKilled, total = info.numTasks)} + completed = stageData.numCompleteTasks, failed = stageData.numFailedTasks, + skipped = 0, reasonToNumKilled = stageData.killedTasksSummary, total = info.numTasks)} {data.inputReadWithUnit} {data.outputWriteWithUnit} @@ -316,7 +319,7 @@ private[ui] class StagePagedTable( } } - private def failureReasonHtml(s: StageInfo): Seq[Node] = { + private def failureReasonHtml(s: v1.StageData): Seq[Node] = { val failureReason = s.failureReason.getOrElse("") val isMultiline = failureReason.indexOf('\n') >= 0 // Display the first line by default @@ -342,7 +345,7 @@ private[ui] class StagePagedTable( {failureReasonSummary}{details} } - private def makeDescription(s: StageInfo, descriptionOption: Option[String]): Seq[Node] = { + private def makeDescription(s: v1.StageData, descriptionOption: Option[String]): Seq[Node] = { val basePathUri = UIUtils.prependBaseUri(basePath) val killLink = if (killEnabled) { @@ -366,8 +369,8 @@ private[ui] class StagePagedTable( val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" val nameLink = {s.name} - val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) - val details = if (s.details.nonEmpty) { + val cachedRddInfos = store.rddList().filter { rdd => s.rddIds.contains(rdd.id) } + val details = if (s.details != null && s.details.nonEmpty) { +details @@ -402,14 +405,14 @@ private[ui] class StagePagedTable( } private[ui] class StageDataSource( - stages: Seq[StageInfo], - listener: JobProgressListener, + store: AppStateStore, + stages: Seq[v1.StageData], currentTime: Long, pageSize: Int, sortColumn: String, desc: Boolean) extends PagedDataSource[StageTableRowData](pageSize) { - // Convert StageInfo to StageTableRowData which contains the final contents to show in the table - // so that we can avoid creating duplicate contents during sorting the data + // Convert v1.StageData to StageTableRowData which contains the final contents to show in the + // table so that we can avoid creating duplicate contents during sorting the data private val data = stages.map(stageRow).sorted(ordering(sortColumn, desc)) private var _slicedStageIds: Set[Int] = _ @@ -422,26 +425,20 @@ private[ui] class StageDataSource( r } - private def stageRow(s: StageInfo): StageTableRowData = { - val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) + private def stageRow(stageData: v1.StageData): StageTableRowData = { + val description = stageData.description.getOrElse("") - if (stageDataOption.isEmpty) { - return new MissingStageTableRowData(s, s.stageId, s.attemptId) - } - val stageData = stageDataOption.get - - val description = stageData.description - - val formattedSubmissionTime = s.submissionTime match { - case Some(t) => UIUtils.formatDate(new Date(t)) + val formattedSubmissionTime = stageData.submissionTime match { + case Some(t) => UIUtils.formatDate(t) case None => "Unknown" } - val finishTime = s.completionTime.getOrElse(currentTime) + val finishTime = stageData.completionTime.map(_.getTime()).getOrElse(currentTime) // The submission time for a stage is misleading because it counts the time // the stage waits to be launched. (SPARK-10930) - val taskLaunchTimes = - stageData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) + val taskLaunchTimes = store + .taskList(stageData.stageId, stageData.attemptId, 0, Int.MaxValue, v1.TaskSorting.ID) + .map(_.launchTime.getTime()).filter(_ > 0) val duration: Option[Long] = if (taskLaunchTimes.nonEmpty) { val startTime = taskLaunchTimes.min @@ -459,20 +456,20 @@ private[ui] class StageDataSource( val inputReadWithUnit = if (inputRead > 0) Utils.bytesToString(inputRead) else "" val outputWrite = stageData.outputBytes val outputWriteWithUnit = if (outputWrite > 0) Utils.bytesToString(outputWrite) else "" - val shuffleRead = stageData.shuffleReadTotalBytes + val shuffleRead = stageData.shuffleReadBytes val shuffleReadWithUnit = if (shuffleRead > 0) Utils.bytesToString(shuffleRead) else "" val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" new StageTableRowData( - s, - stageDataOption, - s.stageId, - s.attemptId, + stageData, + Some(stageData), + stageData.stageId, + stageData.attemptId, stageData.schedulingPool, - description, - s.submissionTime.getOrElse(0), + stageData.description, + stageData.submissionTime.getOrElse(new Date(0)), formattedSubmissionTime, duration.getOrElse(-1), formattedDuration, @@ -494,7 +491,7 @@ private[ui] class StageDataSource( val ordering: Ordering[StageTableRowData] = sortColumn match { case "Stage Id" => Ordering.by(_.stageId) case "Pool Name" => Ordering.by(_.schedulingPool) - case "Description" => Ordering.by(x => (x.descriptionOption, x.stageInfo.name)) + case "Description" => Ordering.by(x => (x.descriptionOption, x.stage.name)) case "Submitted" => Ordering.by(_.submissionTime) case "Duration" => Ordering.by(_.duration) case "Input" => Ordering.by(_.inputRead) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 181465bdf9609..67c29bdb5f987 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -17,36 +17,47 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1.StageStatus import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all stages in the given SparkContext. */ -private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { +private[ui] class StagesTab(val parent: SparkUI, val store: AppStateStore) + extends SparkUITab(parent, "stages") { + val sc = parent.sc val conf = parent.conf val killEnabled = parent.killEnabled - val progressListener = parent.jobProgressListener - val operationGraphListener = parent.operationGraphListener - val executorsListener = parent.executorsListener attachPage(new AllStagesPage(this)) - attachPage(new StagePage(this)) + attachPage(new StagePage(this, store)) attachPage(new PoolPage(this)) - def isFairScheduler: Boolean = progressListener.schedulingMode == Some(SchedulingMode.FAIR) + def isFairScheduler: Boolean = { + store.environmentInfo().sparkProperties.toMap + .get("spark.scheduler.mode") + .map { mode => mode == SchedulingMode.FAIR } + .getOrElse(false) + } def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - val stageId = Option(request.getParameter("id")).map(_.toInt) - stageId.foreach { id => - if (progressListener.activeStages.contains(id)) { - sc.foreach(_.cancelStage(id, "killed via the Web UI")) - // Do a quick pause here to give Spark time to kill the stage so it shows up as - // killed after the refresh. Note that this will block the serving thread so the - // time should be limited in duration. - Thread.sleep(100) + Option(request.getParameter("id")).map(_.toInt).foreach { id => + try { + val status = store.lastStageAttempt(id).status + if (status == StageStatus.ACTIVE || status == StageStatus.PENDING) { + sc.foreach(_.cancelStage(id, "killed via the Web UI")) + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } catch { + case _: NoSuchElementException => } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala deleted file mode 100644 index ac1a74ad8029d..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ /dev/null @@ -1,295 +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.jobs - -import scala.collection.mutable -import scala.collection.mutable.{HashMap, LinkedHashMap} - -import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor._ -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} -import org.apache.spark.util.AccumulatorContext -import org.apache.spark.util.collection.OpenHashSet - -private[spark] object UIData { - - class ExecutorSummary { - var taskTime : Long = 0 - var failedTasks : Int = 0 - var succeededTasks : Int = 0 - var reasonToNumKilled : Map[String, Int] = Map.empty - var inputBytes : Long = 0 - var inputRecords : Long = 0 - var outputBytes : Long = 0 - var outputRecords : Long = 0 - var shuffleRead : Long = 0 - var shuffleReadRecords : Long = 0 - var shuffleWrite : Long = 0 - var shuffleWriteRecords : Long = 0 - var memoryBytesSpilled : Long = 0 - var diskBytesSpilled : Long = 0 - var isBlacklisted : Int = 0 - } - - class JobUIData( - var jobId: Int = -1, - var submissionTime: Option[Long] = None, - var completionTime: Option[Long] = None, - var stageIds: Seq[Int] = Seq.empty, - var jobGroup: Option[String] = None, - var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, - /* Tasks */ - // `numTasks` is a potential underestimate of the true number of tasks that this job will run. - // This may be an underestimate because the job start event references all of the result - // stages' transitive stage dependencies, but some of these stages might be skipped if their - // output is available from earlier runs. - // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. - var numTasks: Int = 0, - var numActiveTasks: Int = 0, - var numCompletedTasks: Int = 0, - var numSkippedTasks: Int = 0, - var numFailedTasks: Int = 0, - var reasonToNumKilled: Map[String, Int] = Map.empty, - /* Stages */ - var numActiveStages: Int = 0, - // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: - var completedStageIndices: mutable.HashSet[Int] = new mutable.HashSet[Int](), - var numSkippedStages: Int = 0, - var numFailedStages: Int = 0 - ) - - class StageUIData { - var numActiveTasks: Int = _ - var numCompleteTasks: Int = _ - var completedIndices = new OpenHashSet[Int]() - var numFailedTasks: Int = _ - var reasonToNumKilled: Map[String, Int] = Map.empty - - var executorRunTime: Long = _ - var executorCpuTime: Long = _ - - var inputBytes: Long = _ - var inputRecords: Long = _ - var outputBytes: Long = _ - var outputRecords: Long = _ - var shuffleReadTotalBytes: Long = _ - var shuffleReadRecords : Long = _ - var shuffleWriteBytes: Long = _ - var shuffleWriteRecords: Long = _ - var memoryBytesSpilled: Long = _ - var diskBytesSpilled: Long = _ - var isBlacklisted: Int = _ - - var schedulingPool: String = "" - var description: Option[String] = None - - var accumulables = new HashMap[Long, AccumulableInfo] - var taskData = new LinkedHashMap[Long, TaskUIData] - var executorSummary = new HashMap[String, ExecutorSummary] - - def hasInput: Boolean = inputBytes > 0 - def hasOutput: Boolean = outputBytes > 0 - def hasShuffleRead: Boolean = shuffleReadTotalBytes > 0 - def hasShuffleWrite: Boolean = shuffleWriteBytes > 0 - def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 && diskBytesSpilled > 0 - } - - /** - * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. - */ - class TaskUIData private( - private var _taskInfo: TaskInfo, - private var _metrics: Option[TaskMetricsUIData]) { - - var errorMessage: Option[String] = None - - def taskInfo: TaskInfo = _taskInfo - - def metrics: Option[TaskMetricsUIData] = _metrics - - def updateTaskInfo(taskInfo: TaskInfo): Unit = { - _taskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) - } - - def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { - _metrics = TaskUIData.toTaskMetricsUIData(metrics) - } - - def taskDuration: Option[Long] = { - if (taskInfo.status == "RUNNING") { - Some(_taskInfo.timeRunning(System.currentTimeMillis)) - } else { - _metrics.map(_.executorRunTime) - } - } - } - - object TaskUIData { - def apply(taskInfo: TaskInfo, metrics: Option[TaskMetrics]): TaskUIData = { - new TaskUIData(dropInternalAndSQLAccumulables(taskInfo), toTaskMetricsUIData(metrics)) - } - - private def toTaskMetricsUIData(metrics: Option[TaskMetrics]): Option[TaskMetricsUIData] = { - metrics.map { m => - TaskMetricsUIData( - executorDeserializeTime = m.executorDeserializeTime, - executorDeserializeCpuTime = m.executorDeserializeCpuTime, - executorRunTime = m.executorRunTime, - executorCpuTime = m.executorCpuTime, - resultSize = m.resultSize, - jvmGCTime = m.jvmGCTime, - resultSerializationTime = m.resultSerializationTime, - memoryBytesSpilled = m.memoryBytesSpilled, - diskBytesSpilled = m.diskBytesSpilled, - peakExecutionMemory = m.peakExecutionMemory, - inputMetrics = InputMetricsUIData(m.inputMetrics), - outputMetrics = OutputMetricsUIData(m.outputMetrics), - shuffleReadMetrics = ShuffleReadMetricsUIData(m.shuffleReadMetrics), - shuffleWriteMetrics = ShuffleWriteMetricsUIData(m.shuffleWriteMetrics)) - } - } - - /** - * We don't need to store internal or SQL accumulables as their values will be shown in other - * places, so drop them to reduce the memory usage. - */ - private[spark] def dropInternalAndSQLAccumulables(taskInfo: TaskInfo): TaskInfo = { - val newTaskInfo = new TaskInfo( - taskId = taskInfo.taskId, - index = taskInfo.index, - attemptNumber = taskInfo.attemptNumber, - launchTime = taskInfo.launchTime, - executorId = taskInfo.executorId, - host = taskInfo.host, - taskLocality = taskInfo.taskLocality, - speculative = taskInfo.speculative - ) - newTaskInfo.gettingResultTime = taskInfo.gettingResultTime - newTaskInfo.setAccumulables(taskInfo.accumulables.filter { - accum => !accum.internal && accum.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) - }) - newTaskInfo.finishTime = taskInfo.finishTime - newTaskInfo.failed = taskInfo.failed - newTaskInfo.killed = taskInfo.killed - newTaskInfo - } - } - - case class TaskMetricsUIData( - executorDeserializeTime: Long, - executorDeserializeCpuTime: Long, - executorRunTime: Long, - executorCpuTime: Long, - resultSize: Long, - jvmGCTime: Long, - resultSerializationTime: Long, - memoryBytesSpilled: Long, - diskBytesSpilled: Long, - peakExecutionMemory: Long, - inputMetrics: InputMetricsUIData, - outputMetrics: OutputMetricsUIData, - shuffleReadMetrics: ShuffleReadMetricsUIData, - shuffleWriteMetrics: ShuffleWriteMetricsUIData) - - case class InputMetricsUIData(bytesRead: Long, recordsRead: Long) - object InputMetricsUIData { - def apply(metrics: InputMetrics): InputMetricsUIData = { - if (metrics.bytesRead == 0 && metrics.recordsRead == 0) { - EMPTY - } else { - new InputMetricsUIData( - bytesRead = metrics.bytesRead, - recordsRead = metrics.recordsRead) - } - } - private val EMPTY = InputMetricsUIData(0, 0) - } - - case class OutputMetricsUIData(bytesWritten: Long, recordsWritten: Long) - object OutputMetricsUIData { - def apply(metrics: OutputMetrics): OutputMetricsUIData = { - if (metrics.bytesWritten == 0 && metrics.recordsWritten == 0) { - EMPTY - } else { - new OutputMetricsUIData( - bytesWritten = metrics.bytesWritten, - recordsWritten = metrics.recordsWritten) - } - } - private val EMPTY = OutputMetricsUIData(0, 0) - } - - case class ShuffleReadMetricsUIData( - remoteBlocksFetched: Long, - localBlocksFetched: Long, - remoteBytesRead: Long, - localBytesRead: Long, - fetchWaitTime: Long, - recordsRead: Long, - totalBytesRead: Long, - totalBlocksFetched: Long) - - object ShuffleReadMetricsUIData { - def apply(metrics: ShuffleReadMetrics): ShuffleReadMetricsUIData = { - if ( - metrics.remoteBlocksFetched == 0 && - metrics.localBlocksFetched == 0 && - metrics.remoteBytesRead == 0 && - metrics.localBytesRead == 0 && - metrics.fetchWaitTime == 0 && - metrics.recordsRead == 0 && - metrics.totalBytesRead == 0 && - metrics.totalBlocksFetched == 0) { - EMPTY - } else { - new ShuffleReadMetricsUIData( - remoteBlocksFetched = metrics.remoteBlocksFetched, - localBlocksFetched = metrics.localBlocksFetched, - remoteBytesRead = metrics.remoteBytesRead, - localBytesRead = metrics.localBytesRead, - fetchWaitTime = metrics.fetchWaitTime, - recordsRead = metrics.recordsRead, - totalBytesRead = metrics.totalBytesRead, - totalBlocksFetched = metrics.totalBlocksFetched - ) - } - } - private val EMPTY = ShuffleReadMetricsUIData(0, 0, 0, 0, 0, 0, 0, 0) - } - - case class ShuffleWriteMetricsUIData( - bytesWritten: Long, - recordsWritten: Long, - writeTime: Long) - - object ShuffleWriteMetricsUIData { - def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { - if (metrics.bytesWritten == 0 && metrics.recordsWritten == 0 && metrics.writeTime == 0) { - EMPTY - } else { - new ShuffleWriteMetricsUIData( - bytesWritten = metrics.bytesWritten, - recordsWritten = metrics.recordsWritten, - writeTime = metrics.writeTime - ) - } - } - private val EMPTY = ShuffleWriteMetricsUIData(0, 0, 0) - } - -} diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 43bfe0aacf35b..cf34a245aa471 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -35,20 +35,20 @@ import org.apache.spark.storage.{RDDInfo, StorageLevel} * nodes and children clusters. Additionally, a graph may also have edges that enter or exit * the graph from nodes that belong to adjacent graphs. */ -private[ui] case class RDDOperationGraph( +private[spark] case class RDDOperationGraph( edges: Seq[RDDOperationEdge], outgoingEdges: Seq[RDDOperationEdge], incomingEdges: Seq[RDDOperationEdge], rootCluster: RDDOperationCluster) /** A node in an RDDOperationGraph. This represents an RDD. */ -private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) +private[spark] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) /** * A directed edge connecting two nodes in an RDDOperationGraph. * This represents an RDD dependency. */ -private[ui] case class RDDOperationEdge(fromId: Int, toId: Int) +private[spark] case class RDDOperationEdge(fromId: Int, toId: Int) /** * A cluster that groups nodes together in an RDDOperationGraph. @@ -56,7 +56,7 @@ private[ui] case class RDDOperationEdge(fromId: Int, toId: Int) * This represents any grouping of RDDs, including operation scopes (e.g. textFile, flatMap), * stages, jobs, or any higher level construct. A cluster may be nested inside of other clusters. */ -private[ui] class RDDOperationCluster(val id: String, private var _name: String) { +private[spark] class RDDOperationCluster(val id: String, private var _name: String) { private val _childNodes = new ListBuffer[RDDOperationNode] private val _childClusters = new ListBuffer[RDDOperationCluster] @@ -92,7 +92,7 @@ private[ui] class RDDOperationCluster(val id: String, private var _name: String) } } -private[ui] object RDDOperationGraph extends Logging { +private[spark] object RDDOperationGraph extends Logging { val STAGE_CLUSTER_PREFIX = "stage_" diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala deleted file mode 100644 index 37a12a8646938..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala +++ /dev/null @@ -1,150 +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.scope - -import scala.collection.mutable - -import org.apache.spark.SparkConf -import org.apache.spark.scheduler._ -import org.apache.spark.ui.SparkUI - -/** - * A SparkListener that constructs a DAG of RDD operations. - */ -private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListener { - - // Note: the fate of jobs and stages are tied. This means when we clean up a job, - // we always clean up all of its stages. Similarly, when we clean up a stage, we - // always clean up its job (and, transitively, other stages in the same job). - private[ui] val jobIdToStageIds = new mutable.HashMap[Int, Seq[Int]] - private[ui] val jobIdToSkippedStageIds = new mutable.HashMap[Int, Seq[Int]] - private[ui] val stageIdToJobId = new mutable.HashMap[Int, Int] - private[ui] val stageIdToGraph = new mutable.HashMap[Int, RDDOperationGraph] - private[ui] val completedStageIds = new mutable.HashSet[Int] - - // Keep track of the order in which these are inserted so we can remove old ones - private[ui] val jobIds = new mutable.ArrayBuffer[Int] - private[ui] val stageIds = new mutable.ArrayBuffer[Int] - - // How many root nodes to retain in DAG Graph - private[ui] val retainedNodes = - conf.getInt("spark.ui.dagGraph.retainedRootRDDs", Int.MaxValue) - - // How many jobs or stages to retain graph metadata for - private val retainedJobs = - conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) - private val retainedStages = - conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) - - /** - * Return the graph metadata for all stages in the given job. - * An empty list is returned if one or more of its stages has been cleaned up. - */ - def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = synchronized { - val skippedStageIds = jobIdToSkippedStageIds.getOrElse(jobId, Seq.empty) - val graphs = jobIdToStageIds.getOrElse(jobId, Seq.empty) - .flatMap { sid => stageIdToGraph.get(sid) } - // Mark any skipped stages as such - graphs.foreach { g => - val stageId = g.rootCluster.id.replaceAll(RDDOperationGraph.STAGE_CLUSTER_PREFIX, "").toInt - if (skippedStageIds.contains(stageId) && !g.rootCluster.name.contains("skipped")) { - g.rootCluster.setName(g.rootCluster.name + " (skipped)") - } - } - graphs - } - - /** Return the graph metadata for the given stage, or None if no such information exists. */ - def getOperationGraphForStage(stageId: Int): Option[RDDOperationGraph] = synchronized { - stageIdToGraph.get(stageId) - } - - /** On job start, construct a RDDOperationGraph for each stage in the job for display later. */ - override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { - val jobId = jobStart.jobId - val stageInfos = jobStart.stageInfos - - jobIds += jobId - jobIdToStageIds(jobId) = jobStart.stageInfos.map(_.stageId).sorted - - stageInfos.foreach { stageInfo => - val stageId = stageInfo.stageId - stageIds += stageId - stageIdToJobId(stageId) = jobId - stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo, retainedNodes) - trimStagesIfNecessary() - } - - trimJobsIfNecessary() - } - - /** Keep track of stages that have completed. */ - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { - val stageId = stageCompleted.stageInfo.stageId - if (stageIdToJobId.contains(stageId)) { - // Note: Only do this if the stage has not already been cleaned up - // Otherwise, we may never clean this stage from `completedStageIds` - completedStageIds += stageCompleted.stageInfo.stageId - } - } - - /** On job end, find all stages in this job that are skipped and mark them as such. */ - override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized { - val jobId = jobEnd.jobId - jobIdToStageIds.get(jobId).foreach { stageIds => - val skippedStageIds = stageIds.filter { sid => !completedStageIds.contains(sid) } - // Note: Only do this if the job has not already been cleaned up - // Otherwise, we may never clean this job from `jobIdToSkippedStageIds` - jobIdToSkippedStageIds(jobId) = skippedStageIds - } - } - - /** Clean metadata for old stages if we have exceeded the number to retain. */ - private def trimStagesIfNecessary(): Unit = { - if (stageIds.size >= retainedStages) { - val toRemove = math.max(retainedStages / 10, 1) - stageIds.take(toRemove).foreach { id => cleanStage(id) } - stageIds.trimStart(toRemove) - } - } - - /** Clean metadata for old jobs if we have exceeded the number to retain. */ - private def trimJobsIfNecessary(): Unit = { - if (jobIds.size >= retainedJobs) { - val toRemove = math.max(retainedJobs / 10, 1) - jobIds.take(toRemove).foreach { id => cleanJob(id) } - jobIds.trimStart(toRemove) - } - } - - /** Clean metadata for the given stage, its job, and all other stages that belong to the job. */ - private[ui] def cleanStage(stageId: Int): Unit = { - completedStageIds.remove(stageId) - stageIdToGraph.remove(stageId) - stageIdToJobId.remove(stageId).foreach { jobId => cleanJob(jobId) } - } - - /** Clean metadata for the given job and all stages that belong to it. */ - private[ui] def cleanJob(jobId: Int): Unit = { - jobIdToSkippedStageIds.remove(jobId) - jobIdToStageIds.remove(jobId).foreach { stageIds => - stageIds.foreach { stageId => cleanStage(stageId) } - } - } - -} 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 a1a0c729b9240..7bbd5efdd68ec 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 @@ -22,13 +22,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} -import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo} -import org.apache.spark.ui.{PagedDataSource, PagedTable, UIUtils, WebUIPage} +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.api.v1.{RDDDataDistribution, RDDPartitionInfo} +import org.apache.spark.ui._ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { - private val listener = parent.listener +private[ui] class RDDPage(parent: SparkUITab, store: AppStateStore) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") @@ -47,11 +47,13 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val blockPrevPageSize = Option(parameterBlockPrevPageSize).map(_.toInt).getOrElse(blockPageSize) val rddId = parameterId.toInt - val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener, includeDetails = true) - .getOrElse { + val rddStorageInfo = try { + store.rdd(rddId) + } catch { + case _: NoSuchElementException => // Rather than crashing, render an "RDD Not Found" page return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) - } + } // Worker table val workerTable = UIUtils.listingTable(workerHeader, workerRow, diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index aa84788f1df88..16d5a4123362c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -19,23 +19,24 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.collection.SortedMap import scala.xml.Node +import org.apache.spark.status.{AppStateStore, StreamBlockData} +import org.apache.spark.status.api.v1 import org.apache.spark.storage._ -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui._ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { - private val listener = parent.listener +private[ui] class StoragePage(parent: SparkUITab, store: AppStateStore) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { - val content = rddTable(listener.rddInfoList) ++ - receiverBlockTables(listener.allExecutorStreamBlockStatus.sortBy(_.executorId)) + val content = rddTable(store.rddList()) ++ receiverBlockTables(store.streamBlocksList()) UIUtils.headerSparkPage("Storage", content, parent) } - private[storage] def rddTable(rdds: Seq[RDDInfo]): Seq[Node] = { + private[storage] def rddTable(rdds: Seq[v1.RDDStorageInfo]): Seq[Node] = { if (rdds.isEmpty) { // Don't show the rdd table if there is no RDD persisted. Nil @@ -57,7 +58,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Size on Disk") /** Render an HTML row representing an RDD */ - private def rddRow(rdd: RDDInfo): Seq[Node] = { + private def rddRow(rdd: v1.RDDStorageInfo): Seq[Node] = { // scalastyle:off @@ -65,35 +66,40 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {rdd.name} - {rdd.storageLevel.description} + {rdd.storageLevel} {rdd.numCachedPartitions.toString} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} - {Utils.bytesToString(rdd.memSize)} - {Utils.bytesToString(rdd.diskSize)} + {Utils.bytesToString(rdd.memoryUsed)} + {Utils.bytesToString(rdd.diskUsed)} // scalastyle:on } - private[storage] def receiverBlockTables(statuses: Seq[ExecutorStreamBlockStatus]): Seq[Node] = { - if (statuses.map(_.numStreamBlocks).sum == 0) { + private[storage] def receiverBlockTables(blocks: Seq[StreamBlockData]): Seq[Node] = { + if (blocks.isEmpty) { // Don't show the tables if there is no stream block Nil } else { - val blocks = statuses.flatMap(_.blocks).groupBy(_.blockId).toSeq.sortBy(_._1.toString) + val sorted = blocks.groupBy(_.name).toSeq.sortBy(_._1.toString)

Receiver Blocks

- {executorMetricsTable(statuses)} - {streamBlockTable(blocks)} + {executorMetricsTable(blocks)} + {streamBlockTable(sorted)}
} } - private def executorMetricsTable(statuses: Seq[ExecutorStreamBlockStatus]): Seq[Node] = { + private def executorMetricsTable(blocks: Seq[StreamBlockData]): Seq[Node] = { + val blockManagers = SortedMap(blocks.groupBy(_.executorId).toSeq: _*) + .map { case (id, blocks) => + new ExecutorStreamSummary(blocks) + } +
Aggregated Block Metrics by Executor
- {UIUtils.listingTable(executorMetricsTableHeader, executorMetricsTableRow, statuses, + {UIUtils.listingTable(executorMetricsTableHeader, executorMetricsTableRow, blockManagers, id = Some("storage-by-executor-stream-blocks"))}
} @@ -105,7 +111,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Total Size on Disk", "Stream Blocks") - private def executorMetricsTableRow(status: ExecutorStreamBlockStatus): Seq[Node] = { + private def executorMetricsTableRow(status: ExecutorStreamSummary): Seq[Node] = { {status.executorId} @@ -125,7 +131,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { } - private def streamBlockTable(blocks: Seq[(BlockId, Seq[BlockUIData])]): Seq[Node] = { + private def streamBlockTable(blocks: Seq[(String, Seq[StreamBlockData])]): Seq[Node] = { if (blocks.isEmpty) { Nil } else { @@ -149,7 +155,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Size") /** Render a stream block */ - private def streamBlockTableRow(block: (BlockId, Seq[BlockUIData])): Seq[Node] = { + private def streamBlockTableRow(block: (String, Seq[StreamBlockData])): Seq[Node] = { val replications = block._2 assert(replications.nonEmpty) // This must be true because it's the result of "groupBy" if (replications.size == 1) { @@ -161,33 +167,36 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { } private def streamBlockTableSubrow( - blockId: BlockId, block: BlockUIData, replication: Int, firstSubrow: Boolean): Seq[Node] = { + blockId: String, + block: StreamBlockData, + replication: Int, + firstSubrow: Boolean): Seq[Node] = { val (storageLevel, size) = streamBlockStorageLevelDescriptionAndSize(block) { if (firstSubrow) { - {block.blockId.toString} + {block.name} {replication.toString} } } - {block.location} + {block.hostPort} {storageLevel} {Utils.bytesToString(size)} } private[storage] def streamBlockStorageLevelDescriptionAndSize( - block: BlockUIData): (String, Long) = { - if (block.storageLevel.useDisk) { + block: StreamBlockData): (String, Long) = { + if (block.useDisk) { ("Disk", block.diskSize) - } else if (block.storageLevel.useMemory && block.storageLevel.deserialized) { + } else if (block.useMemory && block.deserialized) { ("Memory", block.memSize) - } else if (block.storageLevel.useMemory && !block.storageLevel.deserialized) { + } else if (block.useMemory && !block.deserialized) { ("Memory Serialized", block.memSize) } else { throw new IllegalStateException(s"Invalid Storage Level: ${block.storageLevel}") @@ -195,3 +204,17 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { } } + +private class ExecutorStreamSummary(blocks: Seq[StreamBlockData]) { + + def executorId: String = blocks.head.executorId + + def location: String = blocks.head.hostPort + + def totalMemSize: Long = blocks.map(_.memSize).sum + + def totalDiskSize: Long = blocks.map(_.diskSize).sum + + def numStreamBlocks: Int = blocks.size + +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 148efb134e14f..33869d9c21963 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -21,67 +21,14 @@ import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ +import org.apache.spark.status.AppStateStore import org.apache.spark.storage._ import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { - val listener = parent.storageListener +private[ui] class StorageTab(parent: SparkUI, store: AppStateStore) + extends SparkUITab(parent, "storage") { - attachPage(new StoragePage(this)) - attachPage(new RDDPage(this)) -} - -/** - * :: DeveloperApi :: - * A SparkListener that prepares information to be displayed on the BlockManagerUI. - * - * This class is thread-safe (unlike JobProgressListener) - */ -@DeveloperApi -@deprecated("This class will be removed in a future release.", "2.2.0") -class StorageListener(storageStatusListener: StorageStatusListener) extends BlockStatusListener { - - private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing - - def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - - /** Filter RDD info to include only those with cached partitions */ - def rddInfoList: Seq[RDDInfo] = synchronized { - _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - } - - /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ - private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { - val rddIdsToUpdate = updatedBlocks.flatMap { case (bid, _) => bid.asRDDId.map(_.rddId) }.toSet - val rddInfosToUpdate = _rddInfoMap.values.toSeq.filter { s => rddIdsToUpdate.contains(s.id) } - StorageUtils.updateRddInfo(rddInfosToUpdate, activeStorageStatusList) - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { - val rddInfos = stageSubmitted.stageInfo.rddInfos - rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info).name = info.name } - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { - // Remove all partitions that are no longer cached in current completed stage - val completedRddIds = stageCompleted.stageInfo.rddInfos.map(r => r.id).toSet - _rddInfoMap.retain { case (id, info) => - !completedRddIds.contains(id) || info.numCachedPartitions > 0 - } - } - - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { - _rddInfoMap.remove(unpersistRDD.rddId) - } - - override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { - super.onBlockUpdated(blockUpdated) - val blockId = blockUpdated.blockUpdatedInfo.blockId - val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel - val memSize = blockUpdated.blockUpdatedInfo.memSize - val diskSize = blockUpdated.blockUpdatedInfo.diskSize - val blockStatus = BlockStatus(storageLevel, memSize, diskSize) - updateRDDInfo(Seq((blockId, blockStatus))) - } + attachPage(new StoragePage(this, store)) + attachPage(new RDDPage(this, store)) } diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json index 25c4fff77e0ad..dc81d11f15fad 100644 --- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 3, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 162, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", @@ -23,14 +25,18 @@ "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 6, 5 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -49,14 +55,18 @@ "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 1, 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 4338, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", @@ -75,5 +85,7 @@ "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index 0f94e3b255dbc..0ad0ccca5c6e1 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -1,6 +1,33 @@ [ { - "id" : "2", - "hostPort" : "172.22.0.167:51487", + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -8,52 +35,55 @@ "totalCores" : 4, "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 4, - "completedTasks" : 0, - "totalTasks" : 4, - "totalDuration" : 2537, - "totalGCTime" : 88, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, "maxMemory" : 908381388, "executorLogs" : { - "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", - "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" }, "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 } -}, { - "id" : "driver", - "hostPort" : "172.22.0.167:51475", +} ,{ + "id" : "2", + "hostPort" : "172.22.0.167:51487", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, "diskUsed" : 0, - "totalCores" : 0, - "maxTasks" : 0, + "totalCores" : 4, + "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 0, + "failedTasks" : 4, "completedTasks" : 0, - "totalTasks" : 0, - "totalDuration" : 0, - "totalGCTime" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, "maxMemory" : 908381388, - "executorLogs" : { }, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + }, "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 } }, { "id" : "1", @@ -110,39 +140,9 @@ "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" }, "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 - } -}, { - "id" : "3", - "hostPort" : "172.22.0.167:51485", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 4, - "maxTasks" : 4, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 12, - "totalTasks" : 12, - "totalDuration" : 2453, - "totalGCTime" : 72, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : true, - "maxMemory" : 908381388, - "executorLogs" : { - "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", - "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" - }, - "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 0f94e3b255dbc..7727bef0178bd 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -1,6 +1,33 @@ [ { - "id" : "2", - "hostPort" : "172.22.0.167:51487", + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { }, + "memoryMetrics": { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -8,52 +35,55 @@ "totalCores" : 4, "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 4, - "completedTasks" : 0, - "totalTasks" : 4, - "totalDuration" : 2537, - "totalGCTime" : 88, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, "maxMemory" : 908381388, "executorLogs" : { - "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", - "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" }, "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 } }, { - "id" : "driver", - "hostPort" : "172.22.0.167:51475", + "id" : "2", + "hostPort" : "172.22.0.167:51487", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, "diskUsed" : 0, - "totalCores" : 0, - "maxTasks" : 0, + "totalCores" : 4, + "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 0, + "failedTasks" : 4, "completedTasks" : 0, - "totalTasks" : 0, - "totalDuration" : 0, - "totalGCTime" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, "maxMemory" : 908381388, - "executorLogs" : { }, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + }, "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 524288000 } }, { "id" : "1", @@ -115,34 +145,4 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 } -}, { - "id" : "3", - "hostPort" : "172.22.0.167:51485", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 4, - "maxTasks" : 4, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 12, - "totalTasks" : 12, - "totalDuration" : 2453, - "totalGCTime" : 72, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : true, - "maxMemory" : 908381388, - "executorLogs" : { - "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", - "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" - }, - "memoryMetrics": { - "usedOnHeapStorageMemory": 0, - "usedOffHeapStorageMemory": 0, - "totalOnHeapStorageMemory": 384093388, - "totalOffHeapStorageMemory": 524288000 - } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index 92e249c851116..4a8539a8558bb 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -1,6 +1,27 @@ [ { - "id" : "2", - "hostPort" : "172.22.0.111:64539", + "id" : "driver", + "hostPort" : "172.22.0.111:64527", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { } +}, { + "id" : "3", + "hostPort" : "172.22.0.111:64543", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -8,41 +29,44 @@ "totalCores" : 4, "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 6, - "completedTasks" : 0, - "totalTasks" : 6, - "totalDuration" : 2792, - "totalGCTime" : 128, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3457, + "totalGCTime" : 72, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, "executorLogs" : { - "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", - "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" } }, { - "id" : "driver", - "hostPort" : "172.22.0.111:64527", + "id" : "2", + "hostPort" : "172.22.0.111:64539", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, "diskUsed" : 0, - "totalCores" : 0, - "maxTasks" : 0, + "totalCores" : 4, + "maxTasks" : 4, "activeTasks" : 0, - "failedTasks" : 0, + "failedTasks" : 6, "completedTasks" : 0, - "totalTasks" : 0, - "totalDuration" : 0, - "totalGCTime" : 0, + "totalTasks" : 6, + "totalDuration" : 2792, + "totalGCTime" : 128, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, - "executorLogs" : { } + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + } }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -91,28 +115,4 @@ "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" } -}, { - "id" : "3", - "hostPort" : "172.22.0.111:64543", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 4, - "maxTasks" : 4, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 4, - "totalTasks" : 4, - "totalDuration" : 3457, - "totalGCTime" : 72, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 384093388, - "executorLogs" : { - "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", - "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" - } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json index b86ba1e65de12..ae787ee6ccaeb 100644 --- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -2,14 +2,17 @@ "status" : "FAILED", "stageId" : 2, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 7, "numFailedTasks" : 1, + "numKilledTasks" : 0, "executorRunTime" : 278, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:06.296GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", "completionTime" : "2015-02-03T16:43:06.347GMT", + "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -23,5 +26,7 @@ "name" : "count at :20", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 3, 2 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json index c108fa61a4318..987ee065a107d 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json @@ -8,8 +8,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json index c108fa61a4318..987ee065a107d 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json @@ -8,8 +8,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json index 3d7407004d262..063198d3339d0 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json @@ -8,10 +8,12 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 1, "name" : "count at :20", @@ -22,10 +24,12 @@ "numCompletedTasks" : 15, "numSkippedTasks" : 0, "numFailedTasks" : 1, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 1 + "numFailedStages" : 1, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -36,8 +40,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json index 10c7e1c0b36fd..d09ab08cdb445 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json @@ -8,8 +8,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index c2f450ba87c6d..6b3be6bcaebeb 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -23,6 +25,7 @@ "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", + "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], "tasks" : { "8" : { @@ -47,6 +50,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -92,6 +96,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -137,6 +142,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -182,6 +188,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -227,6 +234,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -272,6 +280,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -317,6 +326,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -362,6 +372,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -391,12 +402,18 @@ "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 28000128, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 506859ae545b1..e5fd260cf4ff8 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -23,6 +25,7 @@ "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", + "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], "tasks" : { "8" : { @@ -47,6 +50,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -92,6 +96,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -137,6 +142,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -182,6 +188,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -227,6 +234,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -272,6 +280,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -317,6 +326,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -362,6 +372,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -391,12 +402,18 @@ "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 28000128, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json index 6509df1508b30..a3dda890a94a4 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 3, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 162, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", @@ -23,14 +25,49 @@ "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 6, 5 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } +}, { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "numKilledTasks" : 0, + "executorRunTime" : 278, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:06.296GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", + "completionTime" : "2015-02-03T16:43:06.347GMT", + "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:", + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :20", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "rddIds" : [ 3, 2 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -49,14 +86,18 @@ "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 1, 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 4338, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", @@ -75,31 +116,7 @@ "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "FAILED", - "stageId" : 2, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 7, - "numFailedTasks" : 1, - "executorRunTime" : 278, - "executorCpuTime" : 0, - "submissionTime" : "2015-02-03T16:43:06.296GMT", - "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", - "completionTime" : "2015-02-03T16:43:06.347GMT", - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :20", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] + "rddIds" : [ 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json index 8496863a93469..ca9700d3ccbb7 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 120, "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", @@ -23,9 +25,11 @@ "name" : "foreach at :15", "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", + "rddIds" : [ 0 ], "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", "value" : "5050" - } ] + } ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index f4cec68fbfdf2..c8057c8435f83 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 49294, "recordsRead" : 10000 @@ -64,6 +65,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -108,6 +110,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -152,6 +155,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -196,6 +200,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -240,6 +245,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -284,6 +290,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -328,6 +335,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -372,6 +380,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -416,6 +425,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60489, "recordsRead" : 10000 @@ -460,6 +470,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -504,6 +515,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -548,6 +560,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -592,6 +605,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -636,6 +650,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -680,6 +695,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -724,6 +740,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,6 +785,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -812,6 +830,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -856,6 +875,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index 496a21c328da9..d156654bd1763 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -25,6 +25,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -74,6 +75,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -123,6 +125,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -172,6 +175,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -221,6 +225,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -270,6 +275,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -319,6 +325,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -368,6 +375,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 4328dc753c5d4..6cab51b750abd 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -25,6 +25,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -74,6 +75,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -123,6 +125,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -172,6 +175,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -221,6 +225,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -270,6 +275,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -319,6 +325,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -368,6 +375,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index 8c571430f3a1f..107cc1defc113 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -20,6 +20,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -64,6 +65,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -108,6 +110,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -152,6 +155,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -196,6 +200,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -240,6 +245,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -284,6 +290,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -328,6 +335,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -372,6 +380,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -416,6 +425,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -460,6 +470,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -504,6 +515,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -548,6 +560,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -592,6 +605,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -636,6 +650,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -680,6 +695,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -724,6 +740,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,6 +785,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -812,6 +830,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -856,6 +875,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -900,6 +920,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -944,6 +965,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -988,6 +1010,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1032,6 +1055,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1076,6 +1100,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1120,6 +1145,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1164,6 +1190,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1208,6 +1235,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1252,6 +1280,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1296,6 +1325,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1340,6 +1370,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1384,6 +1415,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1428,6 +1460,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1472,6 +1505,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1516,6 +1550,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1560,6 +1595,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1604,6 +1640,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1648,6 +1685,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1692,6 +1730,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1736,6 +1775,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1780,6 +1820,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1824,6 +1865,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1868,6 +1910,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1912,6 +1955,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1956,6 +2000,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2000,6 +2045,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2044,6 +2090,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2088,6 +2135,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2132,6 +2180,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2176,6 +2225,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index 0bd614bdc756e..05c5fc677a035 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -43,10 +44,10 @@ } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 350, "executorId" : "driver", "host" : "localhost", @@ -55,15 +56,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -82,15 +84,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", "duration" : 350, "executorId" : "driver", "host" : "localhost", @@ -99,15 +101,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -126,15 +129,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", "duration" : 349, "executorId" : "driver", "host" : "localhost", @@ -143,17 +146,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -170,13 +174,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, @@ -193,9 +197,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -214,13 +219,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 3, + "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, @@ -237,9 +242,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -258,15 +264,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", "duration" : 349, "executorId" : "driver", "host" : "localhost", @@ -275,17 +281,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -302,7 +309,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 3842811, "recordsWritten" : 10 } } @@ -328,6 +335,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -372,6 +380,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -416,6 +425,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -460,6 +470,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -504,6 +515,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -527,10 +539,10 @@ } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, "executorId" : "driver", "host" : "localhost", @@ -539,17 +551,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -566,7 +579,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } @@ -592,6 +605,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -615,10 +629,10 @@ } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", "duration" : 84, "executorId" : "driver", "host" : "localhost", @@ -627,17 +641,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 9, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -654,15 +669,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", "duration" : 83, "executorId" : "driver", "host" : "localhost", @@ -671,15 +686,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -698,15 +714,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, "executorId" : "driver", "host" : "localhost", @@ -715,15 +731,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -742,7 +759,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } @@ -768,6 +785,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -812,6 +830,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -835,10 +854,10 @@ } } }, { - "taskId" : 13, - "index" : 13, + "taskId" : 15, + "index" : 15, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.924GMT", + "launchTime" : "2015-05-06T13:03:06.928GMT", "duration" : 76, "executorId" : "driver", "host" : "localhost", @@ -847,7 +866,7 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, @@ -856,6 +875,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -874,7 +894,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95004, + "writeTime" : 602780, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index 0bd614bdc756e..05c5fc677a035 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -43,10 +44,10 @@ } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 350, "executorId" : "driver", "host" : "localhost", @@ -55,15 +56,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -82,15 +84,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", "duration" : 350, "executorId" : "driver", "host" : "localhost", @@ -99,15 +101,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -126,15 +129,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", "duration" : 349, "executorId" : "driver", "host" : "localhost", @@ -143,17 +146,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -170,13 +174,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, @@ -193,9 +197,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -214,13 +219,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 3, + "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, @@ -237,9 +242,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -258,15 +264,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", "duration" : 349, "executorId" : "driver", "host" : "localhost", @@ -275,17 +281,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -302,7 +309,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 3842811, "recordsWritten" : 10 } } @@ -328,6 +335,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -372,6 +380,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -416,6 +425,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -460,6 +470,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -504,6 +515,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -527,10 +539,10 @@ } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, "executorId" : "driver", "host" : "localhost", @@ -539,17 +551,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -566,7 +579,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } @@ -592,6 +605,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -615,10 +629,10 @@ } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", "duration" : 84, "executorId" : "driver", "host" : "localhost", @@ -627,17 +641,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 9, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -654,15 +669,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", "duration" : 83, "executorId" : "driver", "host" : "localhost", @@ -671,15 +686,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -698,15 +714,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, "executorId" : "driver", "host" : "localhost", @@ -715,15 +731,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -742,7 +759,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } @@ -768,6 +785,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -812,6 +830,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -835,10 +854,10 @@ } } }, { - "taskId" : 13, - "index" : 13, + "taskId" : 15, + "index" : 15, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.924GMT", + "launchTime" : "2015-05-06T13:03:06.928GMT", "duration" : 76, "executorId" : "driver", "host" : "localhost", @@ -847,7 +866,7 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, @@ -856,6 +875,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -874,7 +894,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95004, + "writeTime" : 602780, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index b58f1a51ba481..7d670dbc1f6be 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -20,6 +20,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -64,6 +65,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -108,6 +110,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -152,6 +155,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -196,6 +200,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -240,6 +245,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -284,6 +290,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -328,6 +335,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -372,6 +380,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -416,6 +425,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -460,6 +470,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -504,6 +515,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -548,6 +560,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -592,6 +605,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -636,6 +650,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -680,6 +695,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70565, "recordsRead" : 10000 @@ -724,6 +740,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,6 +785,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -812,6 +830,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -856,6 +875,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index a449926ee7dc6..eab3b2d4a4155 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -2,9 +2,11 @@ "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, "executorRunTime" : 120, "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", @@ -23,6 +25,7 @@ "name" : "foreach at :15", "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", + "rddIds" : [ 0 ], "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", @@ -56,6 +59,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -106,6 +110,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -156,6 +161,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -206,6 +212,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -256,6 +263,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -306,6 +314,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -356,6 +365,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -406,6 +416,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -435,12 +446,18 @@ "taskTime" : 418, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 0, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json index 3d7407004d262..063198d3339d0 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json @@ -8,10 +8,12 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 1, "name" : "count at :20", @@ -22,10 +24,12 @@ "numCompletedTasks" : 15, "numSkippedTasks" : 0, "numFailedTasks" : 1, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 1 + "numFailedStages" : 1, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -36,8 +40,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json index 6a9bafd6b2191..e032de41b8e7b 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json @@ -8,10 +8,12 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -22,8 +24,10 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 84f7f1fc8eb09..92665b71e55d5 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -153,7 +153,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex private def testCaching(conf: SparkConf, storageLevel: StorageLevel): Unit = { sc = new SparkContext(conf.setMaster(clusterUrl).setAppName("test")) - sc.jobProgressListener.waitUntilExecutorsUp(2, 30000) + TestUtils.waitUntilExecutorsUp(sc, 2, 30000) val data = sc.parallelize(1 to 1000, 10) val cachedData = data.persist(storageLevel) assert(cachedData.count === 1000) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index fe944031bc948..472952addf353 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -66,7 +66,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. // In this case, we won't receive FetchFailed. And it will make this test fail. // Therefore, we should wait until all slaves are up - sc.jobProgressListener.waitUntilExecutorsUp(2, 60000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val rdd = sc.parallelize(0 until 1000, 10).map(i => (i, 1)).reduceByKey(_ + _) diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 5483f2b8434aa..a15ae040d43a9 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -44,13 +44,13 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont stageIds.size should be(2) val firstStageInfo = eventually(timeout(10 seconds)) { - sc.statusTracker.getStageInfo(stageIds(0)).get + sc.statusTracker.getStageInfo(stageIds.min).get } - firstStageInfo.stageId() should be(stageIds(0)) + firstStageInfo.stageId() should be(stageIds.min) firstStageInfo.currentAttemptId() should be(0) firstStageInfo.numTasks() should be(2) eventually(timeout(10 seconds)) { - val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds(0)).get + val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds.min).get updatedFirstStageInfo.numCompletedTasks() should be(2) updatedFirstStageInfo.numActiveTasks() should be(0) updatedFirstStageInfo.numFailedTasks() should be(0) diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 46f9ac6b0273a..159629825c677 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -224,7 +224,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test") // Wait until all salves are up try { - _sc.jobProgressListener.waitUntilExecutorsUp(numSlaves, 60000) + TestUtils.waitUntilExecutorsUp(_sc, numSlaves, 60000) _sc } catch { case e: Throwable => diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index 7998e3702c122..35f69d57ba609 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -18,15 +18,11 @@ package org.apache.spark.deploy.history import java.util.{Date, NoSuchElementException} -import javax.servlet.Filter import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.collection.mutable -import scala.collection.mutable.ListBuffer import com.codahale.metrics.Counter -import com.google.common.cache.LoadingCache -import com.google.common.util.concurrent.UncheckedExecutionException import org.eclipse.jetty.servlet.ServletContextHandler import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -39,23 +35,10 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.status.api.v1.{ApplicationAttemptInfo => AttemptInfo, ApplicationInfo} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{Clock, ManualClock, Utils} +import org.apache.spark.util.ManualClock class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar with Matchers { - /** - * subclass with access to the cache internals - * @param retainedApplications number of retained applications - */ - class TestApplicationCache( - operations: ApplicationCacheOperations = new StubCacheOperations(), - retainedApplications: Int, - clock: Clock = new ManualClock(0)) - extends ApplicationCache(operations, retainedApplications, clock) { - - def cache(): LoadingCache[CacheKey, CacheEntry] = appCache - } - /** * Stub cache operations. * The state is kept in a map of [[CacheKey]] to [[CacheEntry]], @@ -77,8 +60,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { logDebug(s"getAppUI($appId, $attemptId)") getAppUICount += 1 - instances.get(CacheKey(appId, attemptId)).map( e => - LoadedAppUI(e.ui, updateProbe(appId, attemptId, e.probeTime))) + instances.get(CacheKey(appId, attemptId)).map { e => e.loadedUI } } override def attachSparkUI( @@ -96,10 +78,9 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar attemptId: Option[String], completed: Boolean, started: Long, - ended: Long, - timestamp: Long): SparkUI = { - val ui = putAppUI(appId, attemptId, completed, started, ended, timestamp) - attachSparkUI(appId, attemptId, ui, completed) + ended: Long): LoadedAppUI = { + val ui = putAppUI(appId, attemptId, completed, started, ended) + attachSparkUI(appId, attemptId, ui.ui, completed) ui } @@ -108,23 +89,12 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar attemptId: Option[String], completed: Boolean, started: Long, - ended: Long, - timestamp: Long): SparkUI = { - val ui = newUI(appId, attemptId, completed, started, ended) - putInstance(appId, attemptId, ui, completed, timestamp) + ended: Long): LoadedAppUI = { + val ui = LoadedAppUI(newUI(appId, attemptId, completed, started, ended)) + instances(CacheKey(appId, attemptId)) = new CacheEntry(ui, completed) ui } - def putInstance( - appId: String, - attemptId: Option[String], - ui: SparkUI, - completed: Boolean, - timestamp: Long): Unit = { - instances += (CacheKey(appId, attemptId) -> - new CacheEntry(ui, completed, updateProbe(appId, attemptId, timestamp), timestamp)) - } - /** * Detach a reconstructed UI * @@ -146,23 +116,6 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar attached.get(CacheKey(appId, attemptId)) } - /** - * The update probe. - * @param appId application to probe - * @param attemptId attempt to probe - * @param updateTime timestamp of this UI load - */ - private[history] def updateProbe( - appId: String, - attemptId: Option[String], - updateTime: Long)(): Boolean = { - updateProbeCount += 1 - logDebug(s"isUpdated($appId, $attemptId, ${updateTime})") - val entry = instances.get(CacheKey(appId, attemptId)).get - val updated = entry.probeTime > updateTime - logDebug(s"entry = $entry; updated = $updated") - updated - } } /** @@ -210,15 +163,13 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar val now = clock.getTimeMillis() // add the entry - operations.putAppUI(app1, None, true, now, now, now) + operations.putAppUI(app1, None, true, now, now) // make sure its local operations.getAppUI(app1, None).get operations.getAppUICount = 0 // now expect it to be found - val cacheEntry = cache.lookupCacheEntry(app1, None) - assert(1 === cacheEntry.probeTime) - assert(cacheEntry.completed) + cache.withSparkUI(app1, None) { _ => } // assert about queries made of the operations assert(1 === operations.getAppUICount, "getAppUICount") assert(1 === operations.attachCount, "attachCount") @@ -236,8 +187,8 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar assert(0 === operations.detachCount, "attachCount") // evict the entry - operations.putAndAttach("2", None, true, time2, time2, time2) - operations.putAndAttach("3", None, true, time2, time2, time2) + operations.putAndAttach("2", None, true, time2, time2) + operations.putAndAttach("3", None, true, time2, time2) cache.get("2") cache.get("3") @@ -248,7 +199,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar val appId = "app1" val attemptId = Some("_01") val time3 = clock.getTimeMillis() - operations.putAppUI(appId, attemptId, false, time3, 0, time3) + operations.putAppUI(appId, attemptId, false, time3, 0) // expect an error here assertNotFound(appId, None) } @@ -256,10 +207,11 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar test("Test that if an attempt ID is set, it must be used in lookups") { val operations = new StubCacheOperations() val clock = new ManualClock(1) - implicit val cache = new ApplicationCache(operations, retainedApplications = 10, clock = clock) + implicit val cache = new ApplicationCache(operations, retainedApplications = 10, + clock = clock) val appId = "app1" val attemptId = Some("_01") - operations.putAppUI(appId, attemptId, false, clock.getTimeMillis(), 0, 0) + operations.putAppUI(appId, attemptId, false, clock.getTimeMillis(), 0) assertNotFound(appId, None) } @@ -271,50 +223,29 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar test("Incomplete apps refreshed") { val operations = new StubCacheOperations() val clock = new ManualClock(50) - val window = 500 - implicit val cache = new ApplicationCache(operations, retainedApplications = 5, clock = clock) + implicit val cache = new ApplicationCache(operations, 5, clock) val metrics = cache.metrics // add the incomplete app // add the entry val started = clock.getTimeMillis() val appId = "app1" val attemptId = Some("001") - operations.putAppUI(appId, attemptId, false, started, 0, started) - val firstEntry = cache.lookupCacheEntry(appId, attemptId) - assert(started === firstEntry.probeTime, s"timestamp in $firstEntry") - assert(!firstEntry.completed, s"entry is complete: $firstEntry") - assertMetric("lookupCount", metrics.lookupCount, 1) + val initialUI = operations.putAndAttach(appId, attemptId, false, started, 0) + val firstUI = cache.withSparkUI(appId, attemptId) { ui => ui } + assertMetric("lookupCount", metrics.lookupCount, 1) assert(0 === operations.updateProbeCount, "expected no update probe on that first get") - val checkTime = window * 2 - clock.setTime(checkTime) - val entry3 = cache.lookupCacheEntry(appId, attemptId) - assert(firstEntry !== entry3, s"updated entry test from $cache") + // Invalidate the first entry to trigger a re-load. + initialUI.invalidate() + + // Update the UI in the stub so that a new one is provided to the cache. + operations.putAppUI(appId, attemptId, true, started, started + 10) + + val updatedUI = cache.withSparkUI(appId, attemptId) { ui => ui } + assert(firstUI !== updatedUI, s"expected updated UI") assertMetric("lookupCount", metrics.lookupCount, 2) - assertMetric("updateProbeCount", metrics.updateProbeCount, 1) - assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 0) - assert(1 === operations.updateProbeCount, s"refresh count in $cache") - assert(0 === operations.detachCount, s"detach count") - assert(entry3.probeTime === checkTime) - - val updateTime = window * 3 - // update the cached value - val updatedApp = operations.putAppUI(appId, attemptId, true, started, updateTime, updateTime) - val endTime = window * 10 - clock.setTime(endTime) - logDebug(s"Before operation = $cache") - val entry5 = cache.lookupCacheEntry(appId, attemptId) - assertMetric("lookupCount", metrics.lookupCount, 3) - assertMetric("updateProbeCount", metrics.updateProbeCount, 2) - // the update was triggered - assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 1) - assert(updatedApp === entry5.ui, s"UI {$updatedApp} did not match entry {$entry5} in $cache") - - // at which point, the refreshes stop - clock.setTime(window * 20) - assertCacheEntryEquals(appId, attemptId, entry5) - assertMetric("updateProbeCount", metrics.updateProbeCount, 2) + assert(1 === operations.detachCount, s"detach count") } /** @@ -337,27 +268,6 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar } } - /** - * Look up the cache entry and assert that it matches in the expected value. - * This assertion works if the two CacheEntries are different -it looks at the fields. - * UI are compared on object equality; the timestamp and completed flags directly. - * @param appId application ID - * @param attemptId attempt ID - * @param expected expected value - * @param cache app cache - */ - def assertCacheEntryEquals( - appId: String, - attemptId: Option[String], - expected: CacheEntry) - (implicit cache: ApplicationCache): Unit = { - val actual = cache.lookupCacheEntry(appId, attemptId) - val errorText = s"Expected get($appId, $attemptId) -> $expected, but got $actual from $cache" - assert(expected.ui === actual.ui, errorText + " SparkUI reference") - assert(expected.completed === actual.completed, errorText + " -completed flag") - assert(expected.probeTime === actual.probeTime, errorText + " -timestamp") - } - /** * Assert that a key wasn't found in cache or loaded. * @@ -370,14 +280,9 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar appId: String, attemptId: Option[String]) (implicit cache: ApplicationCache): Unit = { - val ex = intercept[UncheckedExecutionException] { + val ex = intercept[NoSuchElementException] { cache.get(appId, attemptId) } - var cause = ex.getCause - assert(cause !== null) - if (!cause.isInstanceOf[NoSuchElementException]) { - throw cause - } } test("Large Scale Application Eviction") { @@ -385,12 +290,12 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar val clock = new ManualClock(0) val size = 5 // only two entries are retained, so we expect evictions to occur on lookups - implicit val cache: ApplicationCache = new TestApplicationCache(operations, - retainedApplications = size, clock = clock) + implicit val cache = new ApplicationCache(operations, retainedApplications = size, + clock = clock) val attempt1 = Some("01") - val ids = new ListBuffer[String]() + val ids = new mutable.ListBuffer[String]() // build a list of applications val count = 100 for (i <- 1 to count ) { @@ -398,7 +303,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar ids += appId clock.advance(10) val t = clock.getTimeMillis() - operations.putAppUI(appId, attempt1, true, t, t, t) + operations.putAppUI(appId, attempt1, true, t, t) } // now go through them in sequence reading them, expect evictions ids.foreach { id => @@ -413,20 +318,19 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar test("Attempts are Evicted") { val operations = new StubCacheOperations() - implicit val cache: ApplicationCache = new TestApplicationCache(operations, - retainedApplications = 4) + implicit val cache = new ApplicationCache(operations, 4, new ManualClock()) val metrics = cache.metrics val appId = "app1" val attempt1 = Some("01") val attempt2 = Some("02") val attempt3 = Some("03") - operations.putAppUI(appId, attempt1, true, 100, 110, 110) - operations.putAppUI(appId, attempt2, true, 200, 210, 210) - operations.putAppUI(appId, attempt3, true, 300, 310, 310) + operations.putAppUI(appId, attempt1, true, 100, 110) + operations.putAppUI(appId, attempt2, true, 200, 210) + operations.putAppUI(appId, attempt3, true, 300, 310) val attempt4 = Some("04") - operations.putAppUI(appId, attempt4, true, 400, 410, 410) + operations.putAppUI(appId, attempt4, true, 400, 410) val attempt5 = Some("05") - operations.putAppUI(appId, attempt5, true, 500, 510, 510) + operations.putAppUI(appId, attempt5, true, 500, 510) def expectLoadAndEvictionCounts(expectedLoad: Int, expectedEvictionCount: Int): Unit = { assertMetric("loadCount", metrics.loadCount, expectedLoad) @@ -457,20 +361,14 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar } - test("Instantiate Filter") { - // this is a regression test on the filter being constructable - val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME) - val instance = clazz.newInstance() - instance shouldBe a [Filter] - } - test("redirect includes query params") { - val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME) - val filter = clazz.newInstance().asInstanceOf[ApplicationCacheCheckFilter] - filter.appId = "local-123" + val operations = new StubCacheOperations() + val ui = operations.putAndAttach("foo", None, true, 0, 10) val cache = mock[ApplicationCache] - when(cache.checkForUpdates(any(), any())).thenReturn(true) - ApplicationCacheCheckFilterRelay.setApplicationCache(cache) + when(cache.operations).thenReturn(operations) + val filter = new ApplicationCacheCheckFilter(new CacheKey("foo", None), ui, cache) + ui.invalidate() + val request = mock[HttpServletRequest] when(request.getMethod()).thenReturn("GET") when(request.getRequestURI()).thenReturn("http://localhost:18080/history/local-123/jobs/job/") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 456158d41b93f..d9fb13d8a499e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -37,6 +37,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.scheduler._ @@ -55,6 +56,16 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc Utils.deleteRecursively(testDir) } + private def newProvider( + conf: SparkConf, + clock: Clock = null): FsHistoryProvider = { + if (clock == null) { + new FsHistoryProvider(conf) + } else { + new FsHistoryProvider(conf, clock) + } + } + /** Create a fake log file using the new log format used in Spark 1.3+ */ private def newLogFile( appId: String, @@ -69,7 +80,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("Parse application logs") { val clock = new ManualClock(12345678) - val provider = new FsHistoryProvider(createTestConf(), clock) + val provider = newProvider(createTestConf(), clock) // Write a new-style application log. val newAppComplete = newLogFile("new1", None, inProgress = false) @@ -164,7 +175,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("history file is renamed from inprogress to completed") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, @@ -173,20 +184,18 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc ) updateAndCheck(provider) { list => list.size should be (1) - list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should - endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should endWith(EventLoggingListener.IN_PROGRESS) } logFile1.renameTo(newLogFile("app1", None, inProgress = false)) updateAndCheck(provider) { list => list.size should be (1) - list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should not - endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should not endWith(EventLoggingListener.IN_PROGRESS) } } test("Parse logs that application is not started") { - val provider = new FsHistoryProvider((createTestConf())) + val provider = newProvider(createTestConf()) val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, @@ -198,7 +207,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("SPARK-5582: empty log directory") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, @@ -214,7 +223,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("apps with multiple attempts with order") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = true) writeFile(attempt1, true, None, @@ -275,7 +284,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("log cleaner") { val maxAge = TimeUnit.SECONDS.toMillis(10) val clock = new ManualClock(maxAge / 2) - val provider = new FsHistoryProvider( + val provider = newProvider( createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) val log1 = newLogFile("app1", Some("attempt1"), inProgress = false) @@ -321,7 +330,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) val maxAge = TimeUnit.SECONDS.toMillis(40) val clock = new ManualClock(0) - val provider = new FsHistoryProvider( + val provider = newProvider( createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) val log1 = newLogFile("inProgressApp1", None, inProgress = true) @@ -343,23 +352,29 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc provider.checkForLogs() // This should not trigger any cleanup - updateAndCheck(provider)(list => list.size should be(2)) + updateAndCheck(provider) { list => + list.size should be(2) + } // Should trigger cleanup for first file but not second one clock.setTime(firstFileModifiedTime + maxAge + 1) - updateAndCheck(provider)(list => list.size should be(1)) + updateAndCheck(provider) { list => + list.size should be(1) + } assert(!log1.exists()) assert(log2.exists()) // Should cleanup the second file as well. clock.setTime(secondFileModifiedTime + maxAge + 1) - updateAndCheck(provider)(list => list.size should be(0)) + updateAndCheck(provider) { list => + list.size should be(0) + } assert(!log1.exists()) assert(!log2.exists()) } test("Event log copy") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) val logs = (1 to 2).map { i => val log = newLogFile("downloadApp1", Some(s"attempt$i"), inProgress = false) writeFile(log, true, None, @@ -394,7 +409,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("SPARK-8372: new logs with no app ID are ignored") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) // Write a new log file without an app id, to make sure it's ignored. val logFile1 = newLogFile("app1", None, inProgress = true) @@ -408,7 +423,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("provider correctly checks whether fs is in safe mode") { - val provider = spy(new FsHistoryProvider(createTestConf())) + val provider = spy(newProvider(createTestConf())) val dfs = mock(classOf[DistributedFileSystem]) // Asserts that safe mode is false because we can't really control the return value of the mock, // since the API is different between hadoop 1 and 2. @@ -480,7 +495,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc SparkListenerApplicationEnd(5L) ) - val provider = new FsHistoryProvider(createTestConf()) + val provider = newProvider(createTestConf()) updateAndCheck(provider) { list => list.size should be (1) list(0).name should be ("real-app") @@ -497,7 +512,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc var provider: FsHistoryProvider = null try { - provider = new FsHistoryProvider(conf) + provider = newProvider(conf) val log = newLogFile("app1", Some("attempt1"), inProgress = false) writeFile(log, true, None, SparkListenerApplicationStart("app1", Some("app1"), System.currentTimeMillis(), @@ -583,6 +598,41 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + test("invalidate cached UI") { + val storeDir = Utils.createTempDir() + val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) + val provider = newProvider(conf) + val appId = "new1" + + // Write an incomplete app log. + val appLog = newLogFile(appId, None, inProgress = true) + writeFile(appLog, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", None) + ) + provider.checkForLogs() + + // Load the app UI. + val oldUI = provider.getAppUI(appId, None) + assert(oldUI.isDefined) + intercept[NoSuchElementException] { + oldUI.get.ui.store.job(0) + } + + // Add more info to the app log, and trigger the provider to update things. + writeFile(appLog, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", None), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerApplicationEnd(5L) + ) + provider.checkForLogs() + + // Load the UI again and make sure we can get the new info added to the logs. + val freshUI = provider.getAppUI(appId, None) + assert(freshUI.isDefined) + assert(freshUI != oldUI) + freshUI.get.ui.store.job(0) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: @@ -625,7 +675,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } private def createTestConf(): SparkConf = { - new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set(LOCAL_STORE_DIR, Utils.createTempDir().getAbsolutePath()) } private class SafeModeTestProvider(conf: SparkConf, clock: Clock) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 95acb9a54440f..86606f2813225 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -23,6 +23,7 @@ import java.util.zip.ZipInputStream import javax.servlet._ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} +import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps @@ -43,8 +44,9 @@ import org.scalatest.mock.MockitoSugar import org.scalatest.selenium.WebBrowser import org.apache.spark._ +import org.apache.spark.deploy.history.config._ +import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.JobUIData import org.apache.spark.util.{ResetSystemProperties, Utils} /** @@ -64,16 +66,20 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers private val logDir = getTestResourcePath("spark-events") private val expRoot = getTestResourceFile("HistoryServerExpectations") + private val storeDir = Utils.createTempDir(namePrefix = "history") private var provider: FsHistoryProvider = null private var server: HistoryServer = null private var port: Int = -1 def init(extraConf: (String, String)*): Unit = { + Utils.deleteRecursively(storeDir) + assert(storeDir.mkdir()) val conf = new SparkConf() .set("spark.history.fs.logDirectory", logDir) .set("spark.history.fs.update.interval", "0") .set("spark.testing", "true") + .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() @@ -87,14 +93,13 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers def stop(): Unit = { server.stop() + server = null } before { - init() - } - - after{ - stop() + if (server == null) { + init() + } } val cases = Seq( @@ -256,7 +261,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1") badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND) - badStageAttemptId._3 should be (Some("unknown attempt for stage 1. Found attempts: [0]")) + badStageAttemptId._3 should be (Some("unknown attempt 1 for stage 1.")) val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam") badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND) @@ -290,20 +295,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val uiRoot = "/testwebproxybase" System.setProperty("spark.ui.proxyBase", uiRoot) - server.stop() - - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir) - .set("spark.history.fs.update.interval", "0") - .set("spark.testing", "true") - - provider = new FsHistoryProvider(conf) - provider.checkForLogs() - val securityManager = HistoryServer.createSecurityManager(conf) - - server = new HistoryServer(conf, provider, securityManager, 18080) - server.initialize() - server.bind() + stop() + init() val port = server.boundPort @@ -372,7 +365,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } test("incomplete apps get refreshed") { - implicit val webDriver: WebDriver = new HtmlUnitDriver implicit val formats = org.json4s.DefaultFormats @@ -382,12 +374,14 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // a new conf is used with the background thread set and running at its fastest // allowed refresh rate (1Hz) + stop() val myConf = new SparkConf() .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) .set("spark.eventLog.dir", logDir.getAbsolutePath) .set("spark.history.fs.update.interval", "1s") .set("spark.eventLog.enabled", "true") .set("spark.history.cache.window", "250ms") + .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .remove("spark.testing") val provider = new FsHistoryProvider(myConf) val securityManager = HistoryServer.createSecurityManager(myConf) @@ -413,9 +407,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } - // stop the server with the old config, and start the new one - server.stop() - server = new HistoryServer(myConf, provider, securityManager, 18080) + server = new HistoryServer(myConf, provider, securityManager, 0) server.initialize() server.bind() val port = server.boundPort @@ -461,7 +453,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers rootAppPage should not be empty def getAppUI: SparkUI = { - provider.getAppUI(appId, None).get.ui + server.withSparkUI(appId, None) { ui => ui } } // selenium isn't that useful on failures...add our own reporting @@ -503,12 +495,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } - def completedJobs(): Seq[JobUIData] = { - getAppUI.jobProgressListener.completedJobs + def completedJobs(): Seq[JobData] = { + getAppUI.store.jobsList(List(JobExecutionStatus.SUCCEEDED).asJava) } - def activeJobs(): Seq[JobUIData] = { - getAppUI.jobProgressListener.activeJobs.values.toSeq + def activeJobs(): Seq[JobData] = { + getAppUI.store.jobsList(List(JobExecutionStatus.RUNNING).asJava) } activeJobs() should have size 0 @@ -516,7 +508,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers getNumJobs("") should be (1) getNumJobs("/jobs") should be (1) getNumJobsRestful() should be (1) - assert(metrics.lookupCount.getCount > 1, s"lookup count too low in $metrics") + assert(metrics.lookupCount.getCount > 0, s"lookup count too low in $metrics") // dump state before the next bit of test, which is where update // checking really gets stressed 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 a10941b579fe2..13fa7abac524d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2304,13 +2304,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou // OutputCommitCoordinator requires the task info itself to not be null. private def createFakeTaskInfo(): TaskInfo = { val info = new TaskInfo(0, 0, 0, 0L, "", "", TaskLocality.ANY, false) - info.finishTime = 1 // to prevent spurious errors in JobProgressListener + info.finishTime = 1 info } private def createFakeTaskInfoWithId(taskId: Long): TaskInfo = { val info = new TaskInfo(taskId, 0, 0, 0L, "", "", TaskLocality.ANY, false) - info.finishTime = 1 // to prevent spurious errors in JobProgressListener + info.finishTime = 1 info } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala index 9fa8859382911..123f7f49d21b5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo /** @@ -43,7 +43,7 @@ class SparkListenerWithClusterSuite extends SparkFunSuite with LocalSparkContext // This test will check if the number of executors received by "SparkListener" is same as the // number of all executors, so we need to wait until all executors are up - sc.jobProgressListener.waitUntilExecutorsUp(2, 60000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.map(_.toString) diff --git a/core/src/test/scala/org/apache/spark/status/AppStateListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStateListenerSuite.scala new file mode 100644 index 0000000000000..158cd1241ebbe --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/AppStateListenerSuite.scala @@ -0,0 +1,805 @@ +/* + * 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.status + +import java.io.File +import java.lang.{Integer => JInteger, Long => JLong} +import java.util.{Arrays, Date, Properties} + +import scala.collection.JavaConverters._ +import scala.reflect.{classTag, ClassTag} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.kvstore._ +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster._ +import org.apache.spark.status.api.v1 +import org.apache.spark.storage._ +import org.apache.spark.util.Utils + +class AppStateListenerSuite extends SparkFunSuite with BeforeAndAfter { + + private var time: Long = _ + private var testDir: File = _ + private var store: KVStore = _ + + before { + time = 0L + testDir = Utils.createTempDir() + store = KVUtils.open(testDir, getClass().getName()) + } + + after { + store.close() + Utils.deleteRecursively(testDir) + } + + test("environment info") { + val listener = new AppStateListener(store, new SparkConf()) + + val details = Map( + "JVM Information" -> Seq( + "Java Version" -> sys.props("java.version"), + "Java Home" -> sys.props("java.home"), + "Scala Version" -> scala.util.Properties.versionString + ), + "Spark Properties" -> Seq( + "spark.conf.1" -> "1", + "spark.conf.2" -> "2" + ), + "System Properties" -> Seq( + "sys.prop.1" -> "1", + "sys.prop.2" -> "2" + ), + "Classpath Entries" -> Seq( + "/jar1" -> "System", + "/jar2" -> "User" + ) + ) + + listener.onEnvironmentUpdate(SparkListenerEnvironmentUpdate(details)) + + val appEnvKey = classOf[ApplicationEnvironmentInfoWrapper].getName() + check[ApplicationEnvironmentInfoWrapper](appEnvKey) { env => + val info = env.info + + val runtimeInfo = Map(details("JVM Information"): _*) + assert(info.runtime.javaVersion == runtimeInfo("Java Version")) + assert(info.runtime.javaHome == runtimeInfo("Java Home")) + assert(info.runtime.scalaVersion == runtimeInfo("Scala Version")) + + assert(info.sparkProperties === details("Spark Properties")) + assert(info.systemProperties === details("System Properties")) + assert(info.classpathEntries === details("Classpath Entries")) + } + } + + test("scheduler events") { + val listener = new AppStateListener(store, new SparkConf()) + + // Start the application. + time += 1 + listener.onApplicationStart(SparkListenerApplicationStart( + "name", + Some("id"), + time, + "user", + Some("attempt"), + None)) + + check[ApplicationInfoWrapper]("id") { app => + assert(app.info.name === "name") + assert(app.info.id === "id") + assert(app.info.attempts.size === 1) + + val attempt = app.info.attempts.head + assert(attempt.attemptId === Some("attempt")) + assert(attempt.startTime === new Date(time)) + assert(attempt.lastUpdated === new Date(time)) + assert(attempt.endTime === AppStateListener.DEFAULT_DATE) + assert(attempt.sparkUser === "user") + assert(!attempt.completed) + } + + // Start a couple of executors. + time += 1 + val execIds = Array("1", "2") + + execIds.foreach { id => + listener.onExecutorAdded(SparkListenerExecutorAdded(time, id, + new ExecutorInfo(s"$id.example.com", 1, Map()))) + } + + execIds.foreach { id => + check[ExecutorSummaryWrapper](id) { exec => + assert(exec.info.id === id) + assert(exec.info.hostPort === s"$id.example.com") + assert(exec.info.isActive) + } + } + + // Start a job with 2 stages / 4 tasks each + time += 1 + val stages = Seq( + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), + new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2")) + + val stageProps = new Properties() + stageProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup") + stageProps.setProperty("spark.scheduler.pool", "schedPool") + + listener.onJobStart(SparkListenerJobStart(1, time, stages, null)) + + check[JobDataWrapper](1) { job => + assert(job.info.jobId === 1) + assert(job.info.name === stages.last.name) + assert(job.info.description === None) + assert(job.info.status === JobExecutionStatus.RUNNING) + assert(job.info.submissionTime === Some(new Date(time))) + } + + stages.foreach { info => + check[StageDataWrapper](key(info)) { stage => + assert(stage.info.status === v1.StageStatus.PENDING) + assert(stage.jobIds === Set(1)) + } + } + + // Submit stage 1 + time += 1 + stages.head.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stages.head, stageProps)) + + check[JobDataWrapper](1) { job => + assert(job.info.numActiveStages === 1) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.status === v1.StageStatus.ACTIVE) + assert(stage.info.submissionTime === Some(new Date(stages.head.submissionTime.get))) + assert(stage.info.numTasks === stages.head.numTasks) + } + + // Start tasks from stage 1 + time += 1 + var _taskIdTracker = -1L + def nextTaskId(): Long = { + _taskIdTracker += 1 + _taskIdTracker + } + + def createTasks(count: Int, time: Long): Seq[TaskInfo] = { + (1 to count).map { id => + val exec = execIds(id.toInt % execIds.length) + val taskId = nextTaskId() + new TaskInfo(taskId, taskId.toInt, 1, time, exec, s"$exec.example.com", + TaskLocality.PROCESS_LOCAL, id % 2 == 0) + } + } + + val s1Tasks = createTasks(4, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, task)) + } + + assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size) + + check[JobDataWrapper](1) { job => + assert(job.info.numActiveTasks === s1Tasks.size) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numActiveTasks === s1Tasks.size) + assert(stage.info.firstTaskLaunchedTime === Some(new Date(s1Tasks.head.launchTime))) + } + + s1Tasks.foreach { task => + check[TaskDataWrapper](task.taskId) { wrapper => + assert(wrapper.info.taskId === task.taskId) + assert(wrapper.stageId === stages.head.stageId) + assert(wrapper.stageAttemptId === stages.head.attemptId) + assert(Arrays.equals(wrapper.stage, Array(stages.head.stageId, stages.head.attemptId))) + + val runtime = Array[AnyRef](stages.head.stageId: JInteger, stages.head.attemptId: JInteger, + -1L: JLong) + assert(Arrays.equals(wrapper.runtime, runtime)) + + assert(wrapper.info.index === task.index) + assert(wrapper.info.attempt === task.attemptNumber) + assert(wrapper.info.launchTime === new Date(task.launchTime)) + assert(wrapper.info.executorId === task.executorId) + assert(wrapper.info.host === task.host) + assert(wrapper.info.status === task.status) + assert(wrapper.info.taskLocality === task.taskLocality.toString()) + assert(wrapper.info.speculative === task.speculative) + } + } + + // Send executor metrics update. Only update one metric to avoid a lot of boilerplate code. + s1Tasks.foreach { task => + val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED), + Some(1L), None, true, false, None) + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( + task.executorId, + Seq((task.taskId, stages.head.stageId, stages.head.attemptId, Seq(accum))))) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.memoryBytesSpilled === s1Tasks.size) + + val it = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage").first(stage.id) + .closeableIterator() + try { + val execs = it.asScala.takeWhile { v => Arrays.equals(v.stage, stage.id) }.toList + assert(execs.size > 0) + execs.foreach { exec => + assert(exec.info.memoryBytesSpilled === s1Tasks.size / 2) + } + } finally { + it.close() + } + } + + // Fail one of the tasks, re-start it. + time += 1 + s1Tasks.head.finishTime = time + s1Tasks.head.failed = true + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId, + "taskType", TaskResultLost, s1Tasks.head, null)) + + time += 1 + val reattempt = newAttempt(s1Tasks.head, nextTaskId()) + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, + reattempt)) + + assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size + 1) + + check[JobDataWrapper](1) { job => + assert(job.info.numFailedTasks === 1) + assert(job.info.numActiveTasks === s1Tasks.size) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numFailedTasks === 1) + assert(stage.info.numActiveTasks === s1Tasks.size) + } + + check[TaskDataWrapper](s1Tasks.head.taskId) { task => + assert(task.info.status === s1Tasks.head.status) + assert(task.info.duration === Some(s1Tasks.head.duration)) + assert(task.info.errorMessage == Some(TaskResultLost.toErrorString)) + } + + check[TaskDataWrapper](reattempt.taskId) { task => + assert(task.info.index === s1Tasks.head.index) + assert(task.info.attempt === reattempt.attemptNumber) + } + + // Kill one task, restart it. + time += 1 + val killed = s1Tasks.drop(1).head + killed.finishTime = time + killed.failed = true + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId, + "taskType", TaskKilled("killed"), killed, null)) + + check[JobDataWrapper](1) { job => + assert(job.info.numKilledTasks === 1) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numKilledTasks === 1) + } + + check[TaskDataWrapper](killed.taskId) { task => + assert(task.info.index === killed.index) + assert(task.info.errorMessage === Some("killed")) + } + + val reattempt2 = newAttempt(killed, nextTaskId()) + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, + reattempt2)) + + // Succeed all tasks in stage 1. + val pending = s1Tasks.drop(2) ++ Seq(reattempt, reattempt2) + + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + time += 1 + pending.foreach { task => + task.finishTime = time + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId, + "taskType", Success, task, s1Metrics)) + } + + check[JobDataWrapper](1) { job => + assert(job.info.numFailedTasks === 1) + assert(job.info.numKilledTasks === 1) + assert(job.info.numActiveTasks === 0) + assert(job.info.numCompletedTasks === pending.size) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numFailedTasks === 1) + assert(stage.info.numKilledTasks === 1) + assert(stage.info.numActiveTasks === 0) + assert(stage.info.numCompleteTasks === pending.size) + } + + pending.foreach { task => + check[TaskDataWrapper](task.taskId) { wrapper => + assert(wrapper.info.errorMessage === None) + assert(wrapper.info.taskMetrics.get.executorCpuTime === 2L) + val runtime = Array[AnyRef](stages.head.stageId: JInteger, stages.head.attemptId: JInteger, + 4L: JLong) + assert(Arrays.equals(wrapper.runtime, runtime)) + } + } + + assert(store.count(classOf[TaskDataWrapper]) === pending.size + 2) + + // End stage 1. + time += 1 + stages.head.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stages.head)) + + check[JobDataWrapper](1) { job => + assert(job.info.numActiveStages === 0) + assert(job.info.numCompletedStages === 1) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.status === v1.StageStatus.COMPLETE) + assert(stage.info.numFailedTasks === 1) + assert(stage.info.numActiveTasks === 0) + assert(stage.info.numCompleteTasks === pending.size) + } + + // Submit stage 2. + time += 1 + stages.last.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stages.last, stageProps)) + + check[JobDataWrapper](1) { job => + assert(job.info.numActiveStages === 1) + } + + check[StageDataWrapper](key(stages.last)) { stage => + assert(stage.info.status === v1.StageStatus.ACTIVE) + assert(stage.info.submissionTime === Some(new Date(stages.last.submissionTime.get))) + } + + // Start and fail all tasks of stage 2. + time += 1 + val s2Tasks = createTasks(4, time) + s2Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId, stages.last.attemptId, task)) + } + + time += 1 + s2Tasks.foreach { task => + task.finishTime = time + task.failed = true + listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptId, + "taskType", TaskResultLost, task, null)) + } + + check[JobDataWrapper](1) { job => + assert(job.info.numFailedTasks === 1 + s2Tasks.size) + assert(job.info.numActiveTasks === 0) + } + + check[StageDataWrapper](key(stages.last)) { stage => + assert(stage.info.numFailedTasks === s2Tasks.size) + assert(stage.info.numActiveTasks === 0) + } + + // Fail stage 2. + time += 1 + stages.last.completionTime = Some(time) + stages.last.failureReason = Some("uh oh") + listener.onStageCompleted(SparkListenerStageCompleted(stages.last)) + + check[JobDataWrapper](1) { job => + assert(job.info.numCompletedStages === 1) + assert(job.info.numFailedStages === 1) + } + + check[StageDataWrapper](key(stages.last)) { stage => + assert(stage.info.status === v1.StageStatus.FAILED) + assert(stage.info.numFailedTasks === s2Tasks.size) + assert(stage.info.numActiveTasks === 0) + assert(stage.info.numCompleteTasks === 0) + assert(stage.info.failureReason === stages.last.failureReason) + } + + // - Re-submit stage 2, all tasks, and succeed them and the stage. + val oldS2 = stages.last + val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptId + 1, oldS2.name, oldS2.numTasks, + oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics) + + time += 1 + newS2.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(newS2, stageProps)) + assert(store.count(classOf[StageDataWrapper]) === 3) + + val newS2Tasks = createTasks(4, time) + + newS2Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptId, task)) + } + + time += 1 + newS2Tasks.foreach { task => + task.finishTime = time + listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptId, "taskType", Success, + task, null)) + } + + time += 1 + newS2.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(newS2)) + + check[JobDataWrapper](1) { job => + assert(job.info.numActiveStages === 0) + assert(job.info.numFailedStages === 1) + assert(job.info.numCompletedStages === 2) + } + + check[StageDataWrapper](key(newS2)) { stage => + assert(stage.info.status === v1.StageStatus.COMPLETE) + assert(stage.info.numActiveTasks === 0) + assert(stage.info.numCompleteTasks === newS2Tasks.size) + } + + // End job. + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) + + check[JobDataWrapper](1) { job => + assert(job.info.status === JobExecutionStatus.SUCCEEDED) + } + + // Submit a second job that re-uses stage 1 and stage 2. Stage 1 won't be re-run, but + // stage 2 will. This should cause stage 1 to be "skipped" in the new job, and the + // stage 2 re-execution should not change the stats of the already finished job. + time += 1 + listener.onJobStart(SparkListenerJobStart(2, time, Seq(stages.head, newS2), null)) + assert(store.count(classOf[JobDataWrapper]) === 2) + + // The new stage attempt is submitted by the DAGScheduler in a real app, so it's not yet + // known by the listener. Make sure that's the case. + val j2s2 = new StageInfo(newS2.stageId, newS2.attemptId + 1, newS2.name, newS2.numTasks, + newS2.rddInfos, newS2.parentIds, newS2.details, newS2.taskMetrics) + intercept[NoSuchElementException] { + check[StageDataWrapper](key(j2s2)) { _ => () } + } + + time += 1 + j2s2.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(j2s2, stageProps)) + assert(store.count(classOf[StageDataWrapper]) === 4) + + time += 1 + val j2s2Tasks = createTasks(4, time) + + j2s2Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(j2s2.stageId, j2s2.attemptId, task)) + } + + time += 1 + j2s2Tasks.foreach { task => + task.finishTime = time + listener.onTaskEnd(SparkListenerTaskEnd(j2s2.stageId, j2s2.attemptId, "taskType", Success, + task, null)) + } + + time += 1 + j2s2.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(j2s2)) + + time += 1 + listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded)) + + check[JobDataWrapper](1) { job => + assert(job.info.numCompletedStages === 2) + assert(job.info.numCompletedTasks === s1Tasks.size + s2Tasks.size) + } + + check[JobDataWrapper](2) { job => + assert(job.info.status === JobExecutionStatus.SUCCEEDED) + assert(job.info.numCompletedStages === 1) + assert(job.info.numCompletedTasks === j2s2Tasks.size) + assert(job.info.numSkippedStages === 1) + assert(job.info.numSkippedTasks === s1Tasks.size) + } + + // Blacklist an executor. + time += 1 + listener.onExecutorBlacklisted(SparkListenerExecutorBlacklisted(time, "1", 42)) + check[ExecutorSummaryWrapper]("1") { exec => + assert(exec.info.isBlacklisted) + } + + time += 1 + listener.onExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(time, "1")) + check[ExecutorSummaryWrapper]("1") { exec => + assert(!exec.info.isBlacklisted) + } + + // Blacklist a node. + time += 1 + listener.onNodeBlacklisted(SparkListenerNodeBlacklisted(time, "1.example.com", 2)) + check[ExecutorSummaryWrapper]("1") { exec => + assert(exec.info.isBlacklisted) + } + + time += 1 + listener.onNodeUnblacklisted(SparkListenerNodeUnblacklisted(time, "1.example.com")) + check[ExecutorSummaryWrapper]("1") { exec => + assert(!exec.info.isBlacklisted) + } + + // Stop executors. + listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "1", "Test")) + listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "2", "Test")) + + Seq("1", "2").foreach { id => + check[ExecutorSummaryWrapper](id) { exec => + assert(exec.info.id === id) + assert(!exec.info.isActive) + } + } + + // End the application. + listener.onApplicationEnd(SparkListenerApplicationEnd(42L)) + + check[ApplicationInfoWrapper]("id") { app => + assert(app.info.name === "name") + assert(app.info.id === "id") + assert(app.info.attempts.size === 1) + + val attempt = app.info.attempts.head + assert(attempt.attemptId === Some("attempt")) + assert(attempt.startTime === new Date(1L)) + assert(attempt.lastUpdated === new Date(42L)) + assert(attempt.endTime === new Date(42L)) + assert(attempt.duration === 41L) + assert(attempt.sparkUser === "user") + assert(attempt.completed) + } + } + + test("storage events") { + val listener = new AppStateListener(store, new SparkConf()) + val maxMemory = 42L + + // Register a couple of block managers. + val bm1 = BlockManagerId("1", "1.example.com", 42) + val bm2 = BlockManagerId("2", "2.example.com", 84) + Seq(bm1, bm2).foreach { bm => + listener.onExecutorAdded(SparkListenerExecutorAdded(1L, bm.executorId, + new ExecutorInfo(bm.host, 1, Map()))) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm, maxMemory)) + check[ExecutorSummaryWrapper](bm.executorId) { exec => + assert(exec.info.maxMemory === maxMemory) + } + } + + val rdd1b1 = RDDBlockId(1, 1) + val level = StorageLevel.MEMORY_AND_DISK + + // Submit a stage and make sure the RDD is recorded. + val rddInfo = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, Nil) + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rddInfo), Nil, "details1") + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.name === rddInfo.name) + assert(wrapper.info.numPartitions === rddInfo.numPartitions) + assert(wrapper.info.storageLevel === rddInfo.storageLevel.description) + } + + // Add partition 1 replicated on two block managers. + listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b1, level, 1L, 1L))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 1L) + assert(wrapper.info.memoryUsed === 1L) + assert(wrapper.info.diskUsed === 1L) + + assert(wrapper.info.dataDistribution.isDefined) + assert(wrapper.info.dataDistribution.get.size === 1) + + val dist = wrapper.info.dataDistribution.get.head + assert(dist.address === bm1.hostPort) + assert(dist.memoryUsed === 1L) + assert(dist.diskUsed === 1L) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + assert(wrapper.info.partitions.isDefined) + assert(wrapper.info.partitions.get.size === 1) + + val part = wrapper.info.partitions.get.head + assert(part.blockName === rdd1b1.name) + assert(part.storageLevel === level.description) + assert(part.memoryUsed === 1L) + assert(part.diskUsed === 1L) + assert(part.executors === Seq(bm1.executorId)) + } + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 1L) + assert(exec.info.memoryUsed === 1L) + assert(exec.info.diskUsed === 1L) + } + + listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm2, rdd1b1, level, 1L, 1L))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 1L) + assert(wrapper.info.memoryUsed === 2L) + assert(wrapper.info.diskUsed === 2L) + assert(wrapper.info.dataDistribution.get.size === 2L) + assert(wrapper.info.partitions.get.size === 1L) + + val dist = wrapper.info.dataDistribution.get.find(_.address == bm2.hostPort).get + assert(dist.memoryUsed === 1L) + assert(dist.diskUsed === 1L) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + val part = wrapper.info.partitions.get(0) + assert(part.memoryUsed === 2L) + assert(part.diskUsed === 2L) + assert(part.executors === Seq(bm1.executorId, bm2.executorId)) + } + + check[ExecutorSummaryWrapper](bm2.executorId) { exec => + assert(exec.info.rddBlocks === 1L) + assert(exec.info.memoryUsed === 1L) + assert(exec.info.diskUsed === 1L) + } + + // Add a second partition only to bm 1. + val rdd1b2 = RDDBlockId(1, 2) + listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b2, level, + 3L, 3L))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 2L) + assert(wrapper.info.memoryUsed === 5L) + assert(wrapper.info.diskUsed === 5L) + assert(wrapper.info.dataDistribution.get.size === 2L) + assert(wrapper.info.partitions.get.size === 2L) + + val dist = wrapper.info.dataDistribution.get.find(_.address == bm1.hostPort).get + assert(dist.memoryUsed === 4L) + assert(dist.diskUsed === 4L) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + val part = wrapper.info.partitions.get.find(_.blockName === rdd1b2.name).get + assert(part.storageLevel === level.description) + assert(part.memoryUsed === 3L) + assert(part.diskUsed === 3L) + assert(part.executors === Seq(bm1.executorId)) + } + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 2L) + assert(exec.info.memoryUsed === 4L) + assert(exec.info.diskUsed === 4L) + } + + // Remove block 1 from bm 1. + listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b1, + StorageLevel.NONE, 1L, 1L))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 2L) + assert(wrapper.info.memoryUsed === 4L) + assert(wrapper.info.diskUsed === 4L) + assert(wrapper.info.dataDistribution.get.size === 2L) + assert(wrapper.info.partitions.get.size === 2L) + + val dist = wrapper.info.dataDistribution.get.find(_.address == bm1.hostPort).get + assert(dist.memoryUsed === 3L) + assert(dist.diskUsed === 3L) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + val part = wrapper.info.partitions.get.find(_.blockName === rdd1b1.name).get + assert(part.storageLevel === level.description) + assert(part.memoryUsed === 1L) + assert(part.diskUsed === 1L) + assert(part.executors === Seq(bm2.executorId)) + } + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 1L) + assert(exec.info.memoryUsed === 3L) + assert(exec.info.diskUsed === 3L) + } + + // Remove block 2 from bm 2. This should leave only block 2 info in the store. + listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm2, rdd1b1, + StorageLevel.NONE, 1L, 1L))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 1L) + assert(wrapper.info.memoryUsed === 3L) + assert(wrapper.info.diskUsed === 3L) + assert(wrapper.info.dataDistribution.get.size === 1L) + assert(wrapper.info.partitions.get.size === 1L) + assert(wrapper.info.partitions.get(0).blockName === rdd1b2.name) + } + + check[ExecutorSummaryWrapper](bm2.executorId) { exec => + assert(exec.info.rddBlocks === 0L) + assert(exec.info.memoryUsed === 0L) + assert(exec.info.diskUsed === 0L) + } + + // Unpersist RDD1. + listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd1b1.rddId)) + intercept[NoSuchElementException] { + check[RDDStorageInfoWrapper](rdd1b1.rddId) { _ => () } + } + + // Update a StreamBlock. + val stream1 = StreamBlockId(1, 1L) + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, stream1, level, 1L, 1L))) + + check[StreamBlockData](Array(stream1.name, bm1.executorId)) { stream => + assert(stream.name === stream1.name) + assert(stream.executorId === bm1.executorId) + assert(stream.hostPort === bm1.hostPort) + assert(stream.storageLevel === level.description) + assert(stream.useMemory === level.useMemory) + assert(stream.useDisk === level.useDisk) + assert(stream.deserialized === level.deserialized) + assert(stream.memSize === 1L) + assert(stream.diskSize === 1L) + } + + // Drop a StreamBlock. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, stream1, StorageLevel.NONE, 0L, 0L))) + intercept[NoSuchElementException] { + check[StreamBlockData](stream1.name) { _ => () } + } + } + + private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptId) + + private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = { + val value = store.read(classTag[T].runtimeClass, key).asInstanceOf[T] + fn(value) + } + + private def newAttempt(orig: TaskInfo, nextId: Long): TaskInfo = { + // Task reattempts have a different ID, but the same index as the original. + new TaskInfo(nextId, orig.index, orig.attemptNumber + 1, time, orig.executorId, + s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative) + } + +} diff --git a/core/src/test/scala/org/apache/spark/status/AsyncCachingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AsyncCachingStoreSuite.scala new file mode 100644 index 0000000000000..eaaa144afe716 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/AsyncCachingStoreSuite.scala @@ -0,0 +1,212 @@ +/* + * 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.status + +import java.util.concurrent.locks.ReentrantLock + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.mockito.Matchers.{eq => meq, _} +import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.kvstore._ +import org.apache.spark.status.KVUtils._ + +class AsyncCachingStoreSuite extends SparkFunSuite with BeforeAndAfter { + + import config._ + + private val maxElements = 8 + private val testElement = new TestElement("key1") + + private var mockStore: KVStore = _ + private var store: AsyncCachingStore = _ + + before { + mockStore = mock(classOf[KVStore]) + when(mockStore.read(any(), any())).thenAnswer(new Answer[Any]() { + override def answer(invocationOnMock: InvocationOnMock): Any = { + new TestElement(invocationOnMock.getArguments()(1).asInstanceOf[String]) + } + }) + store = new AsyncCachingStore(mockStore, new SparkConf().set(MAX_CACHED_ELEMENTS, maxElements)) + } + + after { + store.close() + } + + test("cache key") { + val key1 = new CacheKey(classOf[TestElement], "key1") + val key2 = new CacheKey(classOf[TestElement], Array(1, 2)) + val key3 = new CacheKey(classOf[TestElement], Array("1", "2")) + val key4 = new CacheKey(classOf[TestElement], Array(1, 2)) + val key5 = new CacheKey(classOf[TestElement], Array(1, 2, 3)) + val nullKey = new CacheKey(null, null) + + assert(key1 != key2) + assert(key2 != key3) + assert(key3 != key4) + assert(key4 != key5) + assert(key5 != nullKey) + + assert(key2 === key4) + assert(key2.hashCode() === key4.hashCode()) + + // Just make sure null elements don't cause .equals() or .hashCode() to fail. + assert(!nullKey.equals(key4)) + nullKey.hashCode() + } + + test("elements are cached on read") { + read() + verifyRead(1) + read() + verifyRead(1) + } + + test("elements are evicted when cache is full") { + read() + verifyRead(1) + fillCache() + read() + verifyRead(2) + } + + test("elements are updated on write") { + read() + verifyRead(1) + fillCache() + store.write(new TestElement(testElement.key)) + verifyRead(1) + + // Write, then fill the cache with the write thread disabled. The read path should + // get the data from the pending write op. + store.haltWrites() + try { + store.write(new TestElement(testElement.key)) + fillCache() + read() + verifyRead(1) + } finally { + store.resumeWrites() + } + } + + test("elements are invalidated on delete") { + read() + verifyRead(1) + + // Halt the write thread and verify that the read path looks at the pending delete op and + // throws the expected exception. + store.haltWrites() + try { + store.delete(testElement.getClass(), testElement.key) + intercept[NoSuchElementException] { + read() + } + } finally { + store.resumeWrites() + } + + // With writes enabled again, wait for the write thread to drain the delete op and verify the + // read again. The mock always returns an element, so check that a new read was issued. + eventually(timeout(1 second), interval(100 millis)) { + verify(mockStore, times(1)).delete(meq(classOf[TestElement]), meq(testElement.key)) + } + read() + verifyRead(2) + } + + test("writes stop if queue is full") { + val max = MAX_WRITE_QUEUE_SIZE.defaultValue.get + + // Stop writes to the store so that they queue up. + val lock = new ReentrantLock() + lock.lock() + + when(mockStore.write(any())).thenAnswer(new Answer[Unit]() { + override def answer(invocation: InvocationOnMock): Unit = { + lock.lock() + lock.unlock() + } + }) + + var needUnlock = true + try { + var writes = 0 + val writer = new Thread(new Runnable() { + override def run(): Unit = { + // Because the first write will be dequeued and fed to the mock method above, making room + // for an extra element in the queue, we need to enqueue max + 1. + (0 to max).foreach { i => + store.write(new TestElement(i.toString())) + writes += 1 + } + + // This next write should block. The checks below may be a little racy, but they + // shouldn't cause false negatives (i.e., if the code is broken the test might still + // pass, but the test won't fail if the code is correct). + store.write(new TestElement("last")) + } + }) + writer.setDaemon(true) + writer.start() + + eventually(timeout(1 second), interval(10 millis)) { + assert(writes === max + 1) + } + + assert(writer.isAlive()) + + lock.unlock() + needUnlock = false + + eventually(timeout(1 second), interval(10 millis)) { + assert(!writer.isAlive()) + } + + } finally { + if (needUnlock) { + lock.unlock() + } + } + } + + // Fill the cache with dummy elements to make sure old elements are evicted. + private def fillCache(): Unit = { + (1 to maxElements).foreach { i => + store.write(new TestElement(s"filler$i")) + } + } + + private def verifyRead(count: Int): Unit = { + verify(mockStore, times(count)).read(meq(classOf[TestElement]), meq(testElement.key)) + } + + private def read(): TestElement = store.read(classOf[TestElement], testElement.key) + +} + +private case class TestElement(@KVIndexParam val key: String) diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala deleted file mode 100644 index 1bfb0c1547ec4..0000000000000 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ /dev/null @@ -1,62 +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.status.api.v1 - -import java.util.Date - -import scala.collection.mutable.LinkedHashMap - -import org.apache.spark.SparkFunSuite -import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality} -import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} - -class AllStagesResourceSuite extends SparkFunSuite { - - def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { - val tasks = new LinkedHashMap[Long, TaskUIData] - taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => - tasks(idx.toLong) = TaskUIData( - new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None) - } - - val stageUiData = new StageUIData() - stageUiData.taskData = tasks - val status = StageStatus.ACTIVE - val stageInfo = new StageInfo( - 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc") - val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false) - - stageData.firstTaskLaunchedTime - } - - test("firstTaskLaunchedTime when there are no tasks") { - val result = getFirstTaskLaunchTime(Seq()) - assert(result == None) - } - - test("firstTaskLaunchedTime when there are tasks but none launched") { - val result = getFirstTaskLaunchTime(Seq(-100L, -200L, -300L)) - assert(result == None) - } - - test("firstTaskLaunchedTime when there are tasks and some launched") { - val result = getFirstTaskLaunchTime(Seq(-100L, 1449255596000L, 1449255597000L)) - assert(result == Some(new Date(1449255596000L))) - } - -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala deleted file mode 100644 index 06acca3943c20..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala +++ /dev/null @@ -1,113 +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 org.apache.spark.SparkFunSuite -import org.apache.spark.scheduler._ - -class BlockStatusListenerSuite extends SparkFunSuite { - - test("basic functions") { - val blockManagerId = BlockManagerId("0", "localhost", 10000) - val listener = new BlockStatusListener() - - // Add a block manager and a new block status - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(0, blockManagerId, 0)) - listener.onBlockUpdated(SparkListenerBlockUpdated( - BlockUpdatedInfo( - blockManagerId, - StreamBlockId(0, 100), - StorageLevel.MEMORY_AND_DISK, - memSize = 100, - diskSize = 100))) - // The new block status should be added to the listener - val expectedBlock = BlockUIData( - StreamBlockId(0, 100), - "localhost:10000", - StorageLevel.MEMORY_AND_DISK, - memSize = 100, - diskSize = 100 - ) - val expectedExecutorStreamBlockStatus = Seq( - ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) - ) - assert(listener.allExecutorStreamBlockStatus === expectedExecutorStreamBlockStatus) - - // Add the second block manager - val blockManagerId2 = BlockManagerId("1", "localhost", 10001) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(0, blockManagerId2, 0)) - // Add a new replication of the same block id from the second manager - listener.onBlockUpdated(SparkListenerBlockUpdated( - BlockUpdatedInfo( - blockManagerId2, - StreamBlockId(0, 100), - StorageLevel.MEMORY_AND_DISK, - memSize = 100, - diskSize = 100))) - val expectedBlock2 = BlockUIData( - StreamBlockId(0, 100), - "localhost:10001", - StorageLevel.MEMORY_AND_DISK, - memSize = 100, - diskSize = 100 - ) - // Each block manager should contain one block - val expectedExecutorStreamBlockStatus2 = Set( - ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)), - ExecutorStreamBlockStatus("1", "localhost:10001", Seq(expectedBlock2)) - ) - assert(listener.allExecutorStreamBlockStatus.toSet === expectedExecutorStreamBlockStatus2) - - // Remove a replication of the same block - listener.onBlockUpdated(SparkListenerBlockUpdated( - BlockUpdatedInfo( - blockManagerId2, - StreamBlockId(0, 100), - StorageLevel.NONE, // StorageLevel.NONE means removing it - memSize = 0, - diskSize = 0))) - // Only the first block manager contains a block - val expectedExecutorStreamBlockStatus3 = Set( - ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)), - ExecutorStreamBlockStatus("1", "localhost:10001", Seq.empty) - ) - assert(listener.allExecutorStreamBlockStatus.toSet === expectedExecutorStreamBlockStatus3) - - // Remove the second block manager at first but add a new block status - // from this removed block manager - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(0, blockManagerId2)) - listener.onBlockUpdated(SparkListenerBlockUpdated( - BlockUpdatedInfo( - blockManagerId2, - StreamBlockId(0, 100), - StorageLevel.MEMORY_AND_DISK, - memSize = 100, - diskSize = 100))) - // The second block manager is removed so we should not see the new block - val expectedExecutorStreamBlockStatus4 = Seq( - ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) - ) - assert(listener.allExecutorStreamBlockStatus === expectedExecutorStreamBlockStatus4) - - // Remove the last block manager - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(0, blockManagerId)) - // No block manager now so we should dop all block managers - assert(listener.allExecutorStreamBlockStatus.isEmpty) - } - -} diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala deleted file mode 100644 index 9835f11a2f7ed..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ /dev/null @@ -1,167 +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 org.apache.spark.{SparkConf, SparkFunSuite, Success} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler._ - -/** - * Test the behavior of StorageStatusListener in response to all relevant events. - */ -class StorageStatusListenerSuite extends SparkFunSuite { - private val bm1 = BlockManagerId("big", "dog", 1) - private val bm2 = BlockManagerId("fat", "duck", 2) - private val taskInfo1 = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) - private val taskInfo2 = new TaskInfo(0, 0, 0, 0, "fat", "duck", TaskLocality.ANY, false) - private val conf = new SparkConf() - - test("block manager added/removed") { - conf.set("spark.ui.retainedDeadExecutors", "1") - val listener = new StorageStatusListener(conf) - - // Block manager add - assert(listener.executorIdToStorageStatus.size === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - assert(listener.executorIdToStorageStatus.size === 1) - assert(listener.executorIdToStorageStatus.get("big").isDefined) - assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) - assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) - assert(listener.executorIdToStorageStatus.size === 2) - assert(listener.executorIdToStorageStatus.get("fat").isDefined) - assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) - assert(listener.executorIdToStorageStatus("fat").maxMem === 2000L) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - - // Block manager remove - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm1)) - assert(listener.executorIdToStorageStatus.size === 1) - assert(!listener.executorIdToStorageStatus.get("big").isDefined) - assert(listener.executorIdToStorageStatus.get("fat").isDefined) - assert(listener.deadExecutorStorageStatus.size === 1) - assert(listener.deadExecutorStorageStatus(0).blockManagerId.executorId.equals("big")) - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm2)) - assert(listener.executorIdToStorageStatus.size === 0) - assert(!listener.executorIdToStorageStatus.get("big").isDefined) - assert(!listener.executorIdToStorageStatus.get("fat").isDefined) - assert(listener.deadExecutorStorageStatus.size === 1) - assert(listener.deadExecutorStorageStatus(0).blockManagerId.executorId.equals("fat")) - } - - test("task end without updated blocks") { - val listener = new StorageStatusListener(conf) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) - val taskMetrics = new TaskMetrics - - // Task end with no updated blocks - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - } - - test("updated blocks") { - val listener = new StorageStatusListener(conf) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) - - val blockUpdateInfos1 = Seq( - BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), - BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) - ) - val blockUpdateInfos2 = - Seq(BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) - - // Add some new blocks - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - postUpdateBlock(listener, blockUpdateInfos1) - assert(listener.executorIdToStorageStatus("big").numBlocks === 2) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - postUpdateBlock(listener, blockUpdateInfos2) - assert(listener.executorIdToStorageStatus("big").numBlocks === 2) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - - // Dropped the blocks - val droppedBlockInfo1 = Seq( - BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.NONE, 0L, 0L), - BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) - ) - val droppedBlockInfo2 = Seq( - BlockUpdatedInfo(bm2, RDDBlockId(1, 2), StorageLevel.NONE, 0L, 0L), - BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) - ) - - postUpdateBlock(listener, droppedBlockInfo1) - assert(listener.executorIdToStorageStatus("big").numBlocks === 1) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) - assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - postUpdateBlock(listener, droppedBlockInfo2) - assert(listener.executorIdToStorageStatus("big").numBlocks === 1) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - } - - test("unpersist RDD") { - val listener = new StorageStatusListener(conf) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - val blockUpdateInfos1 = Seq( - BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), - BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) - ) - val blockUpdateInfos2 = - Seq(BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) - postUpdateBlock(listener, blockUpdateInfos1) - postUpdateBlock(listener, blockUpdateInfos2) - assert(listener.executorIdToStorageStatus("big").numBlocks === 3) - - // Unpersist RDD - listener.onUnpersistRDD(SparkListenerUnpersistRDD(9090)) - assert(listener.executorIdToStorageStatus("big").numBlocks === 3) - listener.onUnpersistRDD(SparkListenerUnpersistRDD(4)) - assert(listener.executorIdToStorageStatus("big").numBlocks === 2) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) - listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) - assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - } - - private def postUpdateBlock( - listener: StorageStatusListener, updateBlockInfos: Seq[BlockUpdatedInfo]): Unit = { - updateBlockInfos.foreach { updateBlockInfo => - listener.onBlockUpdated(SparkListenerBlockUpdated(updateBlockInfo)) - } - } -} diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 499d47b13d702..cf8a6d1945de7 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -22,15 +22,16 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.mockito.Matchers.anyString import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.exec.ExecutorsListener -import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} -import org.apache.spark.ui.scope.RDDOperationGraphListener +import org.apache.spark.status.AppStateStore +import org.apache.spark.status.config._ +import org.apache.spark.ui.jobs.{StagePage, StagesTab} +import org.apache.spark.util.Utils class StagePageSuite extends SparkFunSuite with LocalSparkContext { @@ -55,37 +56,41 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { * This also runs a dummy stage to populate the page with useful content. */ private def renderStagePage(conf: SparkConf): Seq[Node] = { - val jobListener = new JobProgressListener(conf) - val graphListener = new RDDOperationGraphListener(conf) - val executorsListener = new ExecutorsListener(new StorageStatusListener(conf), conf) - val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) - val request = mock(classOf[HttpServletRequest]) - when(tab.conf).thenReturn(conf) - when(tab.progressListener).thenReturn(jobListener) - when(tab.operationGraphListener).thenReturn(graphListener) - when(tab.executorsListener).thenReturn(executorsListener) - when(tab.appName).thenReturn("testing") - when(tab.headerTabs).thenReturn(Seq.empty) - when(request.getParameter("id")).thenReturn("0") - when(request.getParameter("attempt")).thenReturn("0") - val page = new StagePage(tab) + val bus = new ReplayListenerBus() + // Disable caching so that we always get the most up-to-date data from the store. + val store = AppStateStore.createTempStore(conf.set(MAX_CACHED_ELEMENTS, 0), bus) - // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") - // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness - (1 to 2).foreach { - taskId => - val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) - jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) - taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - val taskMetrics = TaskMetrics.empty - taskMetrics.incPeakExecutionMemory(peakExecutionMemory) - jobListener.onTaskEnd( - SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + try { + val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) + when(tab.store).thenReturn(store) + + val request = mock(classOf[HttpServletRequest]) + when(tab.conf).thenReturn(conf) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(request.getParameter("id")).thenReturn("0") + when(request.getParameter("attempt")).thenReturn("0") + val page = new StagePage(tab, store) + + // Simulate a stage in job progress listener + val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") + // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness + (1 to 2).foreach { + taskId => + val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, + false) + bus.postToAll(SparkListenerStageSubmitted(stageInfo)) + bus.postToAll(SparkListenerTaskStart(0, 0, taskInfo)) + taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) + val taskMetrics = TaskMetrics.empty + taskMetrics.incPeakExecutionMemory(peakExecutionMemory) + bus.postToAll(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + } + bus.postToAll(SparkListenerStageCompleted(stageInfo)) + page.render(request) + } finally { + store.close() } - jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) - page.render(request) } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index bdd148875e38a..bbc888cb5906a 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -529,7 +529,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } } - test("stage & job retention") { + ignore("stage & job retention") { val conf = new SparkConf() .setMaster("local") .setAppName("test") @@ -675,34 +675,36 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) rdd.count() - val stage0 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString - assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + - "label="Stage 0";\n subgraph ")) - assert(stage0.contains("{\n label="parallelize";\n " + - "0 [label="ParallelCollectionRDD [0]")) - assert(stage0.contains("{\n label="map";\n " + - "1 [label="MapPartitionsRDD [1]")) - assert(stage0.contains("{\n label="groupBy";\n " + - "2 [label="MapPartitionsRDD [2]")) - - val stage1 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString - assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + - "label="Stage 1";\n subgraph ")) - assert(stage1.contains("{\n label="groupBy";\n " + - "3 [label="ShuffledRDD [3]")) - assert(stage1.contains("{\n label="map";\n " + - "4 [label="MapPartitionsRDD [4]")) - assert(stage1.contains("{\n label="groupBy";\n " + - "5 [label="MapPartitionsRDD [5]")) - - val stage2 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString - assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + - "label="Stage 2";\n subgraph ")) - assert(stage2.contains("{\n label="groupBy";\n " + - "6 [label="ShuffledRDD [6]")) + eventually(timeout(5 seconds), interval(100 milliseconds)) { + val stage0 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString + assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + + "label="Stage 0";\n subgraph ")) + assert(stage0.contains("{\n label="parallelize";\n " + + "0 [label="ParallelCollectionRDD [0]")) + assert(stage0.contains("{\n label="map";\n " + + "1 [label="MapPartitionsRDD [1]")) + assert(stage0.contains("{\n label="groupBy";\n " + + "2 [label="MapPartitionsRDD [2]")) + + val stage1 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString + assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + + "label="Stage 1";\n subgraph ")) + assert(stage1.contains("{\n label="groupBy";\n " + + "3 [label="ShuffledRDD [3]")) + assert(stage1.contains("{\n label="map";\n " + + "4 [label="MapPartitionsRDD [4]")) + assert(stage1.contains("{\n label="groupBy";\n " + + "5 [label="MapPartitionsRDD [5]")) + + val stage2 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString + assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + + "label="Stage 2";\n subgraph ")) + assert(stage2.contains("{\n label="groupBy";\n " + + "6 [label="ShuffledRDD [6]")) + } } } 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 deleted file mode 100644 index 48be3be81755a..0000000000000 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ /dev/null @@ -1,442 +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.jobs - -import java.util.Properties - -import org.scalatest.Matchers - -import org.apache.spark._ -import org.apache.spark.{LocalSparkContext, SparkConf, Success} -import org.apache.spark.executor._ -import org.apache.spark.scheduler._ -import org.apache.spark.ui.jobs.UIData.TaskUIData -import org.apache.spark.util.{AccumulatorContext, Utils} - -class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { - - val jobSubmissionTime = 1421191042750L - val jobCompletionTime = 1421191296660L - - private def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") - SparkListenerStageSubmitted(stageInfo) - } - - private def createStageEndEvent(stageId: Int, failed: Boolean = false) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") - if (failed) { - stageInfo.failureReason = Some("Failed!") - } - SparkListenerStageCompleted(stageInfo) - } - - private def createJobStartEvent( - jobId: Int, - stageIds: Seq[Int], - jobGroup: Option[String] = None): SparkListenerJobStart = { - val stageInfos = stageIds.map { stageId => - new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") - } - val properties: Option[Properties] = jobGroup.map { groupId => - val props = new Properties() - props.setProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) - props - } - SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos, properties.orNull) - } - - private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { - val result = if (failed) JobFailed(new Exception("dummy failure")) else JobSucceeded - SparkListenerJobEnd(jobId, jobCompletionTime, result) - } - - private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { - val stagesThatWontBeRun = jobId * 200 to jobId * 200 + 10 - val stageIds = jobId * 100 to jobId * 100 + 50 - listener.onJobStart(createJobStartEvent(jobId, stageIds ++ stagesThatWontBeRun)) - for (stageId <- stageIds) { - listener.onStageSubmitted(createStageStartEvent(stageId)) - listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0)) - } - listener.onJobEnd(createJobEndEvent(jobId, shouldFail)) - } - - private def assertActiveJobsStateIsEmpty(listener: JobProgressListener) { - listener.getSizesOfActiveStateTrackingCollections.foreach { case (fieldName, size) => - assert(size === 0, s"$fieldName was not empty") - } - } - - test("test LRU eviction of stages") { - def runWithListener(listener: JobProgressListener) : Unit = { - for (i <- 1 to 50) { - listener.onStageSubmitted(createStageStartEvent(i)) - listener.onStageCompleted(createStageEndEvent(i)) - } - assertActiveJobsStateIsEmpty(listener) - } - val conf = new SparkConf() - conf.set("spark.ui.retainedStages", 5.toString) - var listener = new JobProgressListener(conf) - - // Test with 5 retainedStages - runWithListener(listener) - listener.completedStages.size should be (5) - listener.completedStages.map(_.stageId).toSet should be (Set(50, 49, 48, 47, 46)) - - // Test with 0 retainedStages - conf.set("spark.ui.retainedStages", 0.toString) - listener = new JobProgressListener(conf) - runWithListener(listener) - listener.completedStages.size should be (0) - } - - test("test clearing of stageIdToActiveJobs") { - val conf = new SparkConf() - conf.set("spark.ui.retainedStages", 5.toString) - val listener = new JobProgressListener(conf) - val jobId = 0 - val stageIds = 1 to 50 - // Start a job with 50 stages - listener.onJobStart(createJobStartEvent(jobId, stageIds)) - for (stageId <- stageIds) { - listener.onStageSubmitted(createStageStartEvent(stageId)) - } - listener.stageIdToActiveJobIds.size should be > 0 - - // Complete the stages and job - for (stageId <- stageIds) { - listener.onStageCompleted(createStageEndEvent(stageId, failed = false)) - } - listener.onJobEnd(createJobEndEvent(jobId, false)) - assertActiveJobsStateIsEmpty(listener) - listener.stageIdToActiveJobIds.size should be (0) - } - - test("test clearing of jobGroupToJobIds") { - def runWithListener(listener: JobProgressListener): Unit = { - // Run 50 jobs, each with one stage - for (jobId <- 0 to 50) { - listener.onJobStart(createJobStartEvent(jobId, Seq(0), jobGroup = Some(jobId.toString))) - listener.onStageSubmitted(createStageStartEvent(0)) - listener.onStageCompleted(createStageEndEvent(0, failed = false)) - listener.onJobEnd(createJobEndEvent(jobId, false)) - } - assertActiveJobsStateIsEmpty(listener) - } - val conf = new SparkConf() - conf.set("spark.ui.retainedJobs", 5.toString) - - var listener = new JobProgressListener(conf) - runWithListener(listener) - // This collection won't become empty, but it should be bounded by spark.ui.retainedJobs - listener.jobGroupToJobIds.size should be (5) - - // Test with 0 jobs - conf.set("spark.ui.retainedJobs", 0.toString) - listener = new JobProgressListener(conf) - runWithListener(listener) - listener.jobGroupToJobIds.size should be (0) - } - - test("test LRU eviction of jobs") { - val conf = new SparkConf() - conf.set("spark.ui.retainedStages", 5.toString) - conf.set("spark.ui.retainedJobs", 5.toString) - val listener = new JobProgressListener(conf) - - // Run a bunch of jobs to get the listener into a state where we've exceeded both the - // job and stage retention limits: - for (jobId <- 1 to 10) { - runJob(listener, jobId, shouldFail = false) - } - for (jobId <- 200 to 210) { - runJob(listener, jobId, shouldFail = true) - } - assertActiveJobsStateIsEmpty(listener) - // Snapshot the sizes of various soft- and hard-size-limited collections: - val softLimitSizes = listener.getSizesOfSoftSizeLimitedCollections - val hardLimitSizes = listener.getSizesOfHardSizeLimitedCollections - // Run some more jobs: - for (jobId <- 11 to 50) { - runJob(listener, jobId, shouldFail = false) - // We shouldn't exceed the hard / soft limit sizes after the jobs have finished: - listener.getSizesOfSoftSizeLimitedCollections should be (softLimitSizes) - listener.getSizesOfHardSizeLimitedCollections should be (hardLimitSizes) - } - - listener.completedJobs.size should be (5) - listener.completedJobs.map(_.jobId).toSet should be (Set(50, 49, 48, 47, 46)) - - for (jobId <- 51 to 100) { - runJob(listener, jobId, shouldFail = true) - // We shouldn't exceed the hard / soft limit sizes after the jobs have finished: - listener.getSizesOfSoftSizeLimitedCollections should be (softLimitSizes) - listener.getSizesOfHardSizeLimitedCollections should be (hardLimitSizes) - } - assertActiveJobsStateIsEmpty(listener) - - // Completed and failed jobs each their own size limits, so this should still be the same: - listener.completedJobs.size should be (5) - listener.completedJobs.map(_.jobId).toSet should be (Set(50, 49, 48, 47, 46)) - listener.failedJobs.size should be (5) - listener.failedJobs.map(_.jobId).toSet should be (Set(100, 99, 98, 97, 96)) - } - - test("test executor id to summary") { - val conf = new SparkConf() - val listener = new JobProgressListener(conf) - val taskMetrics = TaskMetrics.empty - val shuffleReadMetrics = taskMetrics.createTempShuffleReadMetrics() - assert(listener.stageIdToData.size === 0) - - // finish this task, should get updated shuffleRead - shuffleReadMetrics.incRemoteBytesRead(1000) - taskMetrics.mergeShuffleReadMetrics() - var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - var task = new ShuffleMapTask(0) - val taskType = Utils.getFormattedClassName(task) - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse((0, 0), fail()) - .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 1000) - - // finish a task with unknown executor-id, nothing should happen - taskInfo = - new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) - taskInfo.finishTime = 1 - task = new ShuffleMapTask(0) - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.size === 1) - - // finish this task, should get updated duration - taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - task = new ShuffleMapTask(0) - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse((0, 0), fail()) - .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 2000) - - // finish this task, should get updated duration - taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - task = new ShuffleMapTask(0) - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse((0, 0), fail()) - .executorSummary.getOrElse("exe-2", fail()).shuffleRead === 1000) - } - - test("test task success vs failure counting for different task end reasons") { - val conf = new SparkConf() - val listener = new JobProgressListener(conf) - val metrics = TaskMetrics.empty - val taskInfo = new TaskInfo(1234L, 0, 3, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - val task = new ShuffleMapTask(0) - val taskType = Utils.getFormattedClassName(task) - - // Go through all the failure cases to make sure we are counting them as failures. - val taskFailedReasons = Seq( - Resubmitted, - new FetchFailed(null, 0, 0, 0, "ignored"), - ExceptionFailure("Exception", "description", null, null, None), - TaskResultLost, - ExecutorLostFailure("0", true, Some("Induced failure")), - UnknownReason) - var failCount = 0 - for (reason <- taskFailedReasons) { - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, reason, taskInfo, metrics)) - failCount += 1 - assert(listener.stageIdToData((task.stageId, 0)).numCompleteTasks === 0) - assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) - } - - // Make sure killed tasks are accounted for correctly. - listener.onTaskEnd( - SparkListenerTaskEnd( - task.stageId, 0, taskType, TaskKilled("test"), taskInfo, metrics)) - assert(listener.stageIdToData((task.stageId, 0)).reasonToNumKilled === Map("test" -> 1)) - - // Make sure we count success as success. - listener.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 1, taskType, Success, taskInfo, metrics)) - assert(listener.stageIdToData((task.stageId, 1)).numCompleteTasks === 1) - assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) - } - - test("test update metrics") { - val conf = new SparkConf() - val listener = new JobProgressListener(conf) - - val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) - val execId = "exe-1" - - def makeTaskMetrics(base: Int): TaskMetrics = { - val taskMetrics = TaskMetrics.registered - val shuffleReadMetrics = taskMetrics.createTempShuffleReadMetrics() - val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics - val inputMetrics = taskMetrics.inputMetrics - val outputMetrics = taskMetrics.outputMetrics - shuffleReadMetrics.incRemoteBytesRead(base + 1) - shuffleReadMetrics.incLocalBytesRead(base + 9) - shuffleReadMetrics.incRemoteBlocksFetched(base + 2) - taskMetrics.mergeShuffleReadMetrics() - shuffleWriteMetrics.incBytesWritten(base + 3) - taskMetrics.setExecutorRunTime(base + 4) - taskMetrics.incDiskBytesSpilled(base + 5) - taskMetrics.incMemoryBytesSpilled(base + 6) - inputMetrics.setBytesRead(base + 7) - outputMetrics.setBytesWritten(base + 8) - taskMetrics - } - - def makeTaskInfo(taskId: Long, finishTime: Int = 0): TaskInfo = { - val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, - false) - taskInfo.finishTime = finishTime - taskInfo - } - - listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1234L))) - listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1235L))) - listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1236L))) - listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1237L))) - - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( - (1234L, 0, 0, makeTaskMetrics(0).accumulators().map(AccumulatorSuite.makeInfo)), - (1235L, 0, 0, makeTaskMetrics(100).accumulators().map(AccumulatorSuite.makeInfo)), - (1236L, 1, 0, makeTaskMetrics(200).accumulators().map(AccumulatorSuite.makeInfo))))) - - var stage0Data = listener.stageIdToData.get((0, 0)).get - var stage1Data = listener.stageIdToData.get((1, 0)).get - assert(stage0Data.shuffleReadTotalBytes == 220) - assert(stage1Data.shuffleReadTotalBytes == 410) - assert(stage0Data.shuffleWriteBytes == 106) - assert(stage1Data.shuffleWriteBytes == 203) - assert(stage0Data.executorRunTime == 108) - assert(stage1Data.executorRunTime == 204) - assert(stage0Data.diskBytesSpilled == 110) - assert(stage1Data.diskBytesSpilled == 205) - assert(stage0Data.memoryBytesSpilled == 112) - assert(stage1Data.memoryBytesSpilled == 206) - assert(stage0Data.inputBytes == 114) - assert(stage1Data.inputBytes == 207) - assert(stage0Data.outputBytes == 116) - assert(stage1Data.outputBytes == 208) - - assert( - stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 2) - assert( - stage0Data.taskData.get(1235L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 102) - assert( - stage1Data.taskData.get(1236L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 202) - - // task that was included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(0, 0, taskType, Success, makeTaskInfo(1234L, 1), - makeTaskMetrics(300))) - // task that wasn't included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, taskType, Success, makeTaskInfo(1237L, 1), - makeTaskMetrics(400))) - - stage0Data = listener.stageIdToData.get((0, 0)).get - stage1Data = listener.stageIdToData.get((1, 0)).get - // Task 1235 contributed (100+1)+(100+9) = 210 shuffle bytes, and task 1234 contributed - // (300+1)+(300+9) = 610 total shuffle bytes, so the total for the stage is 820. - assert(stage0Data.shuffleReadTotalBytes == 820) - // Task 1236 contributed 410 shuffle bytes, and task 1237 contributed 810 shuffle bytes. - assert(stage1Data.shuffleReadTotalBytes == 1220) - assert(stage0Data.shuffleWriteBytes == 406) - assert(stage1Data.shuffleWriteBytes == 606) - assert(stage0Data.executorRunTime == 408) - assert(stage1Data.executorRunTime == 608) - assert(stage0Data.diskBytesSpilled == 410) - assert(stage1Data.diskBytesSpilled == 610) - assert(stage0Data.memoryBytesSpilled == 412) - assert(stage1Data.memoryBytesSpilled == 612) - assert(stage0Data.inputBytes == 414) - assert(stage1Data.inputBytes == 614) - assert(stage0Data.outputBytes == 416) - assert(stage1Data.outputBytes == 616) - assert( - stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 302) - assert( - stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 402) - } - - test("drop internal and sql accumulators") { - val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) - val internalAccum = - AccumulableInfo(id = 1, name = Some("internal"), None, None, true, false, None) - val sqlAccum = AccumulableInfo( - id = 2, - name = Some("sql"), - update = None, - value = None, - internal = false, - countFailedValues = false, - metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) - val userAccum = AccumulableInfo( - id = 3, - name = Some("user"), - update = None, - value = None, - internal = false, - countFailedValues = false, - metadata = None) - taskInfo.setAccumulables(List(internalAccum, sqlAccum, userAccum)) - - val newTaskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) - assert(newTaskInfo.accumulables === Seq(userAccum)) - } - - test("SPARK-19146 drop more elements when stageData.taskData.size > retainedTasks") { - val conf = new SparkConf() - conf.set("spark.ui.retainedTasks", "100") - val taskMetrics = TaskMetrics.empty - taskMetrics.mergeShuffleReadMetrics() - val task = new ShuffleMapTask(0) - val taskType = Utils.getFormattedClassName(task) - - val listener1 = new JobProgressListener(conf) - for (t <- 1 to 101) { - val taskInfo = new TaskInfo(t, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - listener1.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - } - // 101 - math.max(100 / 10, 101 - 100) = 91 - assert(listener1.stageIdToData((task.stageId, task.stageAttemptId)).taskData.size === 91) - - val listener2 = new JobProgressListener(conf) - for (t <- 1 to 150) { - val taskInfo = new TaskInfo(t, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) - taskInfo.finishTime = 1 - listener2.onTaskEnd( - SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) - } - // 150 - math.max(100 / 10, 150 - 100) = 100 - assert(listener2.stageIdToData((task.stageId, task.stageAttemptId)).taskData.size === 100) - } - -} diff --git a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala deleted file mode 100644 index 3fb78da0c7476..0000000000000 --- a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala +++ /dev/null @@ -1,226 +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.scope - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.scheduler._ - -/** - * Tests that this listener populates and cleans up its data structures properly. - */ -class RDDOperationGraphListenerSuite extends SparkFunSuite { - private var jobIdCounter = 0 - private var stageIdCounter = 0 - private val maxRetainedJobs = 10 - private val maxRetainedStages = 10 - private val conf = new SparkConf() - .set("spark.ui.retainedJobs", maxRetainedJobs.toString) - .set("spark.ui.retainedStages", maxRetainedStages.toString) - - test("run normal jobs") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - assert(listener.jobIdToStageIds.isEmpty) - assert(listener.jobIdToSkippedStageIds.isEmpty) - assert(listener.stageIdToJobId.isEmpty) - assert(listener.stageIdToGraph.isEmpty) - assert(listener.completedStageIds.isEmpty) - assert(listener.jobIds.isEmpty) - assert(listener.stageIds.isEmpty) - - // Run a few jobs, but not enough for clean up yet - (1 to 3).foreach { numStages => startJob(numStages, listener) } // start 3 jobs and 6 stages - (0 to 5).foreach { i => endStage(startingStageId + i, listener) } // finish all 6 stages - (0 to 2).foreach { i => endJob(startingJobId + i, listener) } // finish all 3 jobs - - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToStageIds(startingJobId).size === 1) - assert(listener.jobIdToStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToStageIds(startingJobId + 2).size === 3) - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds.values.forall(_.isEmpty)) // no skipped stages - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToJobId(startingStageId) === startingJobId) - assert(listener.stageIdToJobId(startingStageId + 1) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 2) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 3) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 4) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 5) === startingJobId + 2) - assert(listener.stageIdToGraph.size === 6) - assert(listener.completedStageIds.size === 6) - assert(listener.jobIds.size === 3) - assert(listener.stageIds.size === 6) - } - - test("run jobs with skipped stages") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run a few jobs, but not enough for clean up yet - // Leave some stages unfinished so that they are marked as skipped - (1 to 3).foreach { numStages => startJob(numStages, listener) } // start 3 jobs and 6 stages - (4 to 5).foreach { i => endStage(startingStageId + i, listener) } // finish only last 2 stages - (0 to 2).foreach { i => endJob(startingJobId + i, listener) } // finish all 3 jobs - - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds(startingJobId).size === 1) - assert(listener.jobIdToSkippedStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToSkippedStageIds(startingJobId + 2).size === 1) // 2 stages not skipped - assert(listener.completedStageIds.size === 2) - - // The rest should be the same as before - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToStageIds(startingJobId).size === 1) - assert(listener.jobIdToStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToStageIds(startingJobId + 2).size === 3) - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToJobId(startingStageId) === startingJobId) - assert(listener.stageIdToJobId(startingStageId + 1) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 2) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 3) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 4) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 5) === startingJobId + 2) - assert(listener.stageIdToGraph.size === 6) - assert(listener.jobIds.size === 3) - assert(listener.stageIds.size === 6) - } - - test("clean up metadata") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run many jobs and stages to trigger clean up - (1 to 10000).foreach { i => - // Note: this must be less than `maxRetainedStages` - val numStages = i % (maxRetainedStages - 2) + 1 - val startingStageIdForJob = stageIdCounter - val jobId = startJob(numStages, listener) - // End some, but not all, stages that belong to this job - // This is to ensure that we have both completed and skipped stages - (startingStageIdForJob until stageIdCounter) - .filter { i => i % 2 == 0 } - .foreach { i => endStage(i, listener) } - // End all jobs - endJob(jobId, listener) - } - - // Ensure we never exceed the max retained thresholds - assert(listener.jobIdToStageIds.size <= maxRetainedJobs) - assert(listener.jobIdToSkippedStageIds.size <= maxRetainedJobs) - assert(listener.stageIdToJobId.size <= maxRetainedStages) - assert(listener.stageIdToGraph.size <= maxRetainedStages) - assert(listener.completedStageIds.size <= maxRetainedStages) - assert(listener.jobIds.size <= maxRetainedJobs) - assert(listener.stageIds.size <= maxRetainedStages) - - // Also ensure we're actually populating these data structures - // Otherwise the previous group of asserts will be meaningless - assert(listener.jobIdToStageIds.nonEmpty) - assert(listener.jobIdToSkippedStageIds.nonEmpty) - assert(listener.stageIdToJobId.nonEmpty) - assert(listener.stageIdToGraph.nonEmpty) - assert(listener.completedStageIds.nonEmpty) - assert(listener.jobIds.nonEmpty) - assert(listener.stageIds.nonEmpty) - - // Ensure we clean up old jobs and stages, not arbitrary ones - assert(!listener.jobIdToStageIds.contains(startingJobId)) - assert(!listener.jobIdToSkippedStageIds.contains(startingJobId)) - assert(!listener.stageIdToJobId.contains(startingStageId)) - assert(!listener.stageIdToGraph.contains(startingStageId)) - assert(!listener.completedStageIds.contains(startingStageId)) - assert(!listener.stageIds.contains(startingStageId)) - assert(!listener.jobIds.contains(startingJobId)) - } - - test("fate sharing between jobs and stages") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run 3 jobs and 8 stages, finishing all 3 jobs but only 2 stages - startJob(5, listener) - startJob(1, listener) - startJob(2, listener) - (0 until 8).foreach { i => startStage(i + startingStageId, listener) } - endStage(startingStageId + 3, listener) - endStage(startingStageId + 4, listener) - (0 until 3).foreach { i => endJob(i + startingJobId, listener) } - - // First, assert the old stuff - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.stageIdToJobId.size === 8) - assert(listener.stageIdToGraph.size === 8) - assert(listener.completedStageIds.size === 2) - - // Cleaning the third job should clean all of its stages - listener.cleanJob(startingJobId + 2) - assert(listener.jobIdToStageIds.size === 2) - assert(listener.jobIdToSkippedStageIds.size === 2) - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToGraph.size === 6) - assert(listener.completedStageIds.size === 2) - - // Cleaning one of the stages in the first job should clean that job and all of its stages - // Note that we still keep around the last stage because it belongs to a different job - listener.cleanStage(startingStageId) - assert(listener.jobIdToStageIds.size === 1) - assert(listener.jobIdToSkippedStageIds.size === 1) - assert(listener.stageIdToJobId.size === 1) - assert(listener.stageIdToGraph.size === 1) - assert(listener.completedStageIds.size === 0) - } - - /** Start a job with the specified number of stages. */ - private def startJob(numStages: Int, listener: RDDOperationGraphListener): Int = { - assert(numStages > 0, "I will not run a job with 0 stages for you.") - val stageInfos = (0 until numStages).map { _ => - val stageInfo = new StageInfo(stageIdCounter, 0, "s", 0, Seq.empty, Seq.empty, "d") - stageIdCounter += 1 - stageInfo - } - val jobId = jobIdCounter - listener.onJobStart(new SparkListenerJobStart(jobId, 0, stageInfos)) - // Also start all stages that belong to this job - stageInfos.map(_.stageId).foreach { sid => startStage(sid, listener) } - jobIdCounter += 1 - jobId - } - - /** Start the stage specified by the given ID. */ - private def startStage(stageId: Int, listener: RDDOperationGraphListener): Unit = { - val stageInfo = new StageInfo(stageId, 0, "s", 0, Seq.empty, Seq.empty, "d") - listener.onStageSubmitted(new SparkListenerStageSubmitted(stageInfo)) - } - - /** Finish the stage specified by the given ID. */ - private def endStage(stageId: Int, listener: RDDOperationGraphListener): Unit = { - val stageInfo = new StageInfo(stageId, 0, "s", 0, Seq.empty, Seq.empty, "d") - listener.onStageCompleted(new SparkListenerStageCompleted(stageInfo)) - } - - /** Finish the job specified by the given ID. */ - private def endJob(jobId: Int, listener: RDDOperationGraphListener): Unit = { - listener.onJobEnd(new SparkListenerJobEnd(jobId, 0, JobSucceeded)) - } - -} diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala index 350c174e24742..d4107cdbaf7a2 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala @@ -20,39 +20,46 @@ package org.apache.spark.ui.storage import org.mockito.Mockito._ import org.apache.spark.SparkFunSuite +import org.apache.spark.status.StreamBlockData +import org.apache.spark.status.api.v1.RDDStorageInfo import org.apache.spark.storage._ class StoragePageSuite extends SparkFunSuite { val storageTab = mock(classOf[StorageTab]) when(storageTab.basePath).thenReturn("http://localhost:4040") - val storagePage = new StoragePage(storageTab) + val storagePage = new StoragePage(storageTab, null) test("rddTable") { - val rdd1 = new RDDInfo(1, + val rdd1 = new RDDStorageInfo(1, "rdd1", 10, - StorageLevel.MEMORY_ONLY, - Seq.empty) - rdd1.memSize = 100 - rdd1.numCachedPartitions = 10 + 10, + StorageLevel.MEMORY_ONLY.description, + 100L, + 0L, + None, + None) - val rdd2 = new RDDInfo(2, + val rdd2 = new RDDStorageInfo(2, "rdd2", 10, - StorageLevel.DISK_ONLY, - Seq.empty) - rdd2.diskSize = 200 - rdd2.numCachedPartitions = 5 - - val rdd3 = new RDDInfo(3, + 5, + StorageLevel.DISK_ONLY.description, + 0L, + 200L, + None, + None) + + val rdd3 = new RDDStorageInfo(3, "rdd3", 10, - StorageLevel.MEMORY_AND_DISK_SER, - Seq.empty) - rdd3.memSize = 400 - rdd3.diskSize = 500 - rdd3.numCachedPartitions = 10 + 10, + StorageLevel.MEMORY_AND_DISK_SER.description, + 400L, + 500L, + None, + None) val xmlNodes = storagePage.rddTable(Seq(rdd1, rdd2, rdd3)) @@ -90,58 +97,85 @@ class StoragePageSuite extends SparkFunSuite { } test("streamBlockStorageLevelDescriptionAndSize") { - val memoryBlock = BlockUIData(StreamBlockId(0, 0), + val memoryBlock = new StreamBlockData("0", + "0", "localhost:1111", - StorageLevel.MEMORY_ONLY, - memSize = 100, - diskSize = 0) + StorageLevel.MEMORY_ONLY.description, + true, + false, + true, + 100, + 0) assert(("Memory", 100) === storagePage.streamBlockStorageLevelDescriptionAndSize(memoryBlock)) - val memorySerializedBlock = BlockUIData(StreamBlockId(0, 0), + val memorySerializedBlock = new StreamBlockData("0", + "0", "localhost:1111", - StorageLevel.MEMORY_ONLY_SER, + StorageLevel.MEMORY_ONLY_SER.description, + true, + false, + false, memSize = 100, diskSize = 0) assert(("Memory Serialized", 100) === storagePage.streamBlockStorageLevelDescriptionAndSize(memorySerializedBlock)) - val diskBlock = BlockUIData(StreamBlockId(0, 0), + val diskBlock = new StreamBlockData("0", + "0", "localhost:1111", - StorageLevel.DISK_ONLY, - memSize = 0, - diskSize = 100) + StorageLevel.DISK_ONLY.description, + false, + true, + false, + 0, + 100) assert(("Disk", 100) === storagePage.streamBlockStorageLevelDescriptionAndSize(diskBlock)) } test("receiverBlockTables") { val blocksForExecutor0 = Seq( - BlockUIData(StreamBlockId(0, 0), + new StreamBlockData(StreamBlockId(0, 0).name, + "0", "localhost:10000", - StorageLevel.MEMORY_ONLY, - memSize = 100, - diskSize = 0), - BlockUIData(StreamBlockId(1, 1), + StorageLevel.MEMORY_ONLY.description, + true, + false, + true, + 100, + 0), + new StreamBlockData(StreamBlockId(1, 1).name, + "0", "localhost:10000", - StorageLevel.DISK_ONLY, - memSize = 0, - diskSize = 100) + StorageLevel.DISK_ONLY.description, + false, + true, + false, + 0, + 100) ) - val executor0 = ExecutorStreamBlockStatus("0", "localhost:10000", blocksForExecutor0) val blocksForExecutor1 = Seq( - BlockUIData(StreamBlockId(0, 0), + new StreamBlockData(StreamBlockId(0, 0).name, + "1", "localhost:10001", - StorageLevel.MEMORY_ONLY, + StorageLevel.MEMORY_ONLY.description, + true, + false, + true, memSize = 100, diskSize = 0), - BlockUIData(StreamBlockId(1, 1), + new StreamBlockData(StreamBlockId(1, 1).name, + "1", "localhost:10001", - StorageLevel.MEMORY_ONLY_SER, - memSize = 100, - diskSize = 0) + StorageLevel.MEMORY_ONLY_SER.description, + true, + false, + false, + 100, + 0) ) - val executor1 = ExecutorStreamBlockStatus("1", "localhost:10001", blocksForExecutor1) - val xmlNodes = storagePage.receiverBlockTables(Seq(executor0, executor1)) + + val xmlNodes = storagePage.receiverBlockTables(blocksForExecutor0 ++ blocksForExecutor1) val executorTable = (xmlNodes \\ "table")(0) val executorHeaders = Seq( @@ -189,8 +223,6 @@ class StoragePageSuite extends SparkFunSuite { test("empty receiverBlockTables") { assert(storagePage.receiverBlockTables(Seq.empty).isEmpty) - val executor0 = ExecutorStreamBlockStatus("0", "localhost:10000", Seq.empty) - val executor1 = ExecutorStreamBlockStatus("1", "localhost:10001", Seq.empty) - assert(storagePage.receiverBlockTables(Seq(executor0, executor1)).isEmpty) } + } diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala deleted file mode 100644 index f6c8418ba3ac4..0000000000000 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ /dev/null @@ -1,206 +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.storage - -import org.scalatest.BeforeAndAfter - -import org.apache.spark._ -import org.apache.spark.scheduler._ -import org.apache.spark.storage._ - -/** - * Test various functionality in the StorageListener that supports the StorageTab. - */ -class StorageTabSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter { - private var bus: LiveListenerBus = _ - private var storageStatusListener: StorageStatusListener = _ - private var storageListener: StorageListener = _ - private val memAndDisk = StorageLevel.MEMORY_AND_DISK - private val memOnly = StorageLevel.MEMORY_ONLY - private val none = StorageLevel.NONE - private val taskInfo = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) - private val taskInfo1 = new TaskInfo(1, 1, 1, 1, "big", "cat", TaskLocality.ANY, false) - private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly, Seq(10)) - private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly, Seq(10)) - private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk, Seq(10)) - private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk, Seq(10)) - private val bm1 = BlockManagerId("big", "dog", 1) - - before { - val conf = new SparkConf() - sc = new SparkContext("local", "test", conf) - bus = new LiveListenerBus(sc) - storageStatusListener = new StorageStatusListener(conf) - storageListener = new StorageListener(storageStatusListener) - bus.addListener(storageStatusListener) - bus.addListener(storageListener) - } - - test("stage submitted / completed") { - assert(storageListener._rddInfoMap.isEmpty) - assert(storageListener.rddInfoList.isEmpty) - - // 2 RDDs are known, but none are cached - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0, rddInfo1), Seq.empty, "details") - bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) - assert(storageListener._rddInfoMap.size === 2) - assert(storageListener.rddInfoList.isEmpty) - - // 4 RDDs are known, but only 2 are cached - val rddInfo2Cached = rddInfo2 - val rddInfo3Cached = rddInfo3 - rddInfo2Cached.numCachedPartitions = 1 - rddInfo3Cached.numCachedPartitions = 1 - val stageInfo1 = new StageInfo( - 1, 0, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), Seq.empty, "details") - bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) - assert(storageListener._rddInfoMap.size === 4) - assert(storageListener.rddInfoList.size === 2) - - // Submitting RDDInfos with duplicate IDs does nothing - val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY, Seq(10)) - rddInfo0Cached.numCachedPartitions = 1 - val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), Seq.empty, "details") - bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) - assert(storageListener._rddInfoMap.size === 4) - assert(storageListener.rddInfoList.size === 2) - - // We only keep around the RDDs that are cached - bus.postToAll(SparkListenerStageCompleted(stageInfo0)) - assert(storageListener._rddInfoMap.size === 2) - assert(storageListener.rddInfoList.size === 2) - } - - test("unpersist") { - val rddInfo0Cached = rddInfo0 - val rddInfo1Cached = rddInfo1 - rddInfo0Cached.numCachedPartitions = 1 - rddInfo1Cached.numCachedPartitions = 1 - val stageInfo0 = new StageInfo( - 0, 0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), Seq.empty, "details") - bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) - assert(storageListener._rddInfoMap.size === 2) - assert(storageListener.rddInfoList.size === 2) - bus.postToAll(SparkListenerUnpersistRDD(0)) - assert(storageListener._rddInfoMap.size === 1) - assert(storageListener.rddInfoList.size === 1) - bus.postToAll(SparkListenerUnpersistRDD(4)) // doesn't exist - assert(storageListener._rddInfoMap.size === 1) - assert(storageListener.rddInfoList.size === 1) - bus.postToAll(SparkListenerUnpersistRDD(1)) - assert(storageListener._rddInfoMap.size === 0) - assert(storageListener.rddInfoList.size === 0) - } - - test("block update") { - val myRddInfo0 = rddInfo0 - val myRddInfo1 = rddInfo1 - val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo( - 0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), Seq.empty, "details") - bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) - assert(storageListener._rddInfoMap.size === 3) - assert(storageListener.rddInfoList.size === 0) // not cached - assert(!storageListener._rddInfoMap(0).isCached) - assert(!storageListener._rddInfoMap(1).isCached) - assert(!storageListener._rddInfoMap(2).isCached) - - // Some blocks updated - val blockUpdateInfos = Seq( - BlockUpdatedInfo(bm1, RDDBlockId(0, 100), memAndDisk, 400L, 0L), - BlockUpdatedInfo(bm1, RDDBlockId(0, 101), memAndDisk, 0L, 400L), - BlockUpdatedInfo(bm1, RDDBlockId(1, 20), memAndDisk, 0L, 240L) - ) - postUpdateBlocks(bus, blockUpdateInfos) - assert(storageListener._rddInfoMap(0).memSize === 400L) - assert(storageListener._rddInfoMap(0).diskSize === 400L) - assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) - assert(storageListener._rddInfoMap(0).isCached) - assert(storageListener._rddInfoMap(1).memSize === 0L) - assert(storageListener._rddInfoMap(1).diskSize === 240L) - assert(storageListener._rddInfoMap(1).numCachedPartitions === 1) - assert(storageListener._rddInfoMap(1).isCached) - assert(!storageListener._rddInfoMap(2).isCached) - assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) - - // Drop some blocks - val blockUpdateInfos2 = Seq( - BlockUpdatedInfo(bm1, RDDBlockId(0, 100), none, 0L, 0L), - BlockUpdatedInfo(bm1, RDDBlockId(1, 20), none, 0L, 0L), - BlockUpdatedInfo(bm1, RDDBlockId(2, 40), none, 0L, 0L), // doesn't actually exist - BlockUpdatedInfo(bm1, RDDBlockId(4, 80), none, 0L, 0L) // doesn't actually exist - ) - postUpdateBlocks(bus, blockUpdateInfos2) - assert(storageListener._rddInfoMap(0).memSize === 0L) - assert(storageListener._rddInfoMap(0).diskSize === 400L) - assert(storageListener._rddInfoMap(0).numCachedPartitions === 1) - assert(storageListener._rddInfoMap(0).isCached) - assert(!storageListener._rddInfoMap(1).isCached) - assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) - assert(!storageListener._rddInfoMap(2).isCached) - assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) - } - - test("verify StorageTab contains all cached rdds") { - - val rddInfo0 = new RDDInfo(0, "rdd0", 1, memOnly, Seq(4)) - val rddInfo1 = new RDDInfo(1, "rdd1", 1, memOnly, Seq(4)) - val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo0), Seq.empty, "details") - val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfo1), Seq.empty, "details") - val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L)) - val blockUpdateInfos2 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(1, 1), memOnly, 200L, 0L)) - bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) - assert(storageListener.rddInfoList.size === 0) - postUpdateBlocks(bus, blockUpdateInfos1) - assert(storageListener.rddInfoList.size === 1) - bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) - assert(storageListener.rddInfoList.size === 1) - bus.postToAll(SparkListenerStageCompleted(stageInfo0)) - assert(storageListener.rddInfoList.size === 1) - postUpdateBlocks(bus, blockUpdateInfos2) - assert(storageListener.rddInfoList.size === 2) - bus.postToAll(SparkListenerStageCompleted(stageInfo1)) - assert(storageListener.rddInfoList.size === 2) - } - - test("verify StorageTab still contains a renamed RDD") { - val rddInfo = new RDDInfo(0, "original_name", 1, memOnly, Seq(4)) - val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo), Seq.empty, "details") - bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) - val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L)) - postUpdateBlocks(bus, blockUpdateInfos1) - assert(storageListener.rddInfoList.size == 1) - - val newName = "new_name" - val rddInfoRenamed = new RDDInfo(0, newName, 1, memOnly, Seq(4)) - val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfoRenamed), Seq.empty, "details") - bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) - assert(storageListener.rddInfoList.size == 1) - assert(storageListener.rddInfoList.head.name == newName) - } - - private def postUpdateBlocks( - bus: SparkListenerBus, blockUpdateInfos: Seq[BlockUpdatedInfo]): Unit = { - blockUpdateInfos.foreach { blockUpdateInfo => - bus.postToAll(SparkListenerBlockUpdated(blockUpdateInfo)) - } - } -} diff --git a/docs/monitoring.md b/docs/monitoring.md index 3e577c5f36778..6bbd3e45be54e 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -220,6 +220,13 @@ The history server can be configured as follows: Number of threads that will be used by history server to process event logs. + + spark.history.store.path + /var/lib/spark-history + + Local directory where history server will cache application history data. + + Note that in all of these UIs, the tables are sortable by clicking their headers, diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 6c0c3ebcaebf4..c71a71bbdd30b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -144,6 +144,7 @@ List buildClassPath(String appClassPath) throws IOException { if (prependClasses || isTesting) { String scala = getScalaVersion(); List projects = Arrays.asList( + "common/kvstore", "common/network-common", "common/network-shuffle", "common/network-yarn", @@ -162,7 +163,8 @@ List buildClassPath(String appClassPath) throws IOException { "sql/core", "sql/hive", "sql/hive-thriftserver", - "streaming" + "streaming", + "ui" ); if (prependClasses) { if (!isTesting) { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index d50882cb1917e..50dcb2c960ae3 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,17 @@ object MimaExcludes { // Exclude rules for 2.3.x lazy val v23excludes = v22excludes ++ Seq( + //SPARK-18085: Better History Server scalability for many / large applications + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ExecutorSummary.executorLogs"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.getSparkUI"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.env.EnvironmentListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.exec.ExecutorsListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.storage.StorageListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.StorageStatusListener"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorStageSummary.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.JobData.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkStatusTracker.this"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.jobs.JobProgressListener"), // [SPARK-20495][SQL] Add StorageLevel to cacheTable API ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.cacheTable") ) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 1f48d71cc7a2b..d7932727b4bd5 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -86,14 +86,14 @@ This file is divided into 3 sections: - + - + diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala deleted file mode 100644 index 3a51ae609303a..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala +++ /dev/null @@ -1,78 +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.status.api.v1.streaming - -import java.util.{ArrayList => JArrayList, Arrays => JArrays, Date, List => JList} -import javax.ws.rs.{GET, Produces, QueryParam} -import javax.ws.rs.core.MediaType - -import org.apache.spark.status.api.v1.streaming.AllBatchesResource._ -import org.apache.spark.streaming.ui.StreamingJobProgressListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllBatchesResource(listener: StreamingJobProgressListener) { - - @GET - def batchesList(@QueryParam("status") statusParams: JList[BatchStatus]): Seq[BatchInfo] = { - batchInfoList(listener, statusParams).sortBy(- _.batchId) - } -} - -private[v1] object AllBatchesResource { - - def batchInfoList( - listener: StreamingJobProgressListener, - statusParams: JList[BatchStatus] = new JArrayList[BatchStatus]()): Seq[BatchInfo] = { - - listener.synchronized { - val statuses = - if (statusParams.isEmpty) JArrays.asList(BatchStatus.values(): _*) else statusParams - val statusToBatches = Seq( - BatchStatus.COMPLETED -> listener.retainedCompletedBatches, - BatchStatus.QUEUED -> listener.waitingBatches, - BatchStatus.PROCESSING -> listener.runningBatches - ) - - val batchInfos = for { - (status, batches) <- statusToBatches - batch <- batches if statuses.contains(status) - } yield { - val batchId = batch.batchTime.milliseconds - val firstFailureReason = batch.outputOperations.flatMap(_._2.failureReason).headOption - - new BatchInfo( - batchId = batchId, - batchTime = new Date(batchId), - status = status.toString, - batchDuration = listener.batchDuration, - inputSize = batch.numRecords, - schedulingDelay = batch.schedulingDelay, - processingTime = batch.processingDelay, - totalDelay = batch.totalDelay, - numActiveOutputOps = batch.numActiveOutputOp, - numCompletedOutputOps = batch.numCompletedOutputOp, - numFailedOutputOps = batch.numFailedOutputOp, - numTotalOutputOps = batch.outputOperations.size, - firstFailureReason = firstFailureReason - ) - } - - batchInfos - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala deleted file mode 100644 index 0eb649f0e1b72..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala +++ /dev/null @@ -1,66 +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.status.api.v1.streaming - -import java.util.Date -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.status.api.v1.NotFoundException -import org.apache.spark.status.api.v1.streaming.AllOutputOperationsResource._ -import org.apache.spark.streaming.Time -import org.apache.spark.streaming.ui.StreamingJobProgressListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllOutputOperationsResource(listener: StreamingJobProgressListener) { - - @GET - def operationsList(@PathParam("batchId") batchId: Long): Seq[OutputOperationInfo] = { - outputOperationInfoList(listener, batchId).sortBy(_.outputOpId) - } -} - -private[v1] object AllOutputOperationsResource { - - def outputOperationInfoList( - listener: StreamingJobProgressListener, - batchId: Long): Seq[OutputOperationInfo] = { - - listener.synchronized { - listener.getBatchUIData(Time(batchId)) match { - case Some(batch) => - for ((opId, op) <- batch.outputOperations) yield { - val jobIds = batch.outputOpIdSparkJobIdPairs - .filter(_.outputOpId == opId).map(_.sparkJobId).toSeq.sorted - - new OutputOperationInfo( - outputOpId = opId, - name = op.name, - description = op.description, - startTime = op.startTime.map(new Date(_)), - endTime = op.endTime.map(new Date(_)), - duration = op.duration, - failureReason = op.failureReason, - jobIds = jobIds - ) - } - case None => throw new NotFoundException("unknown batch: " + batchId) - } - }.toSeq - } -} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.scala deleted file mode 100644 index 5a276a9236a0f..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.scala +++ /dev/null @@ -1,76 +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.status.api.v1.streaming - -import java.util.Date -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.status.api.v1.streaming.AllReceiversResource._ -import org.apache.spark.streaming.ui.StreamingJobProgressListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllReceiversResource(listener: StreamingJobProgressListener) { - - @GET - def receiversList(): Seq[ReceiverInfo] = { - receiverInfoList(listener).sortBy(_.streamId) - } -} - -private[v1] object AllReceiversResource { - - def receiverInfoList(listener: StreamingJobProgressListener): Seq[ReceiverInfo] = { - listener.synchronized { - listener.receivedRecordRateWithBatchTime.map { case (streamId, eventRates) => - - val receiverInfo = listener.receiverInfo(streamId) - val streamName = receiverInfo.map(_.name) - .orElse(listener.streamName(streamId)).getOrElse(s"Stream-$streamId") - val avgEventRate = - if (eventRates.isEmpty) None else Some(eventRates.map(_._2).sum / eventRates.size) - - val (errorTime, errorMessage, error) = receiverInfo match { - case None => (None, None, None) - case Some(info) => - val someTime = - if (info.lastErrorTime >= 0) Some(new Date(info.lastErrorTime)) else None - val someMessage = - if (info.lastErrorMessage.length > 0) Some(info.lastErrorMessage) else None - val someError = - if (info.lastError.length > 0) Some(info.lastError) else None - - (someTime, someMessage, someError) - } - - new ReceiverInfo( - streamId = streamId, - streamName = streamName, - isActive = receiverInfo.map(_.active), - executorId = receiverInfo.map(_.executorId), - executorHost = receiverInfo.map(_.location), - lastErrorTime = errorTime, - lastErrorMessage = errorMessage, - lastError = error, - avgEventRate = avgEventRate, - eventRates = eventRates - ) - }.toSeq - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala index aea75d5a9c8d0..07d8164e1d2c0 100644 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala @@ -19,24 +19,39 @@ package org.apache.spark.status.api.v1.streaming import javax.ws.rs.{Path, PathParam} -import org.apache.spark.status.api.v1.ApiRequestContext +import org.apache.spark.status.api.v1._ +import org.apache.spark.streaming.ui.StreamingJobProgressListener @Path("/v1") private[v1] class ApiStreamingApp extends ApiRequestContext { @Path("applications/{appId}/streaming") - def getStreamingRoot(@PathParam("appId") appId: String): ApiStreamingRootResource = { - withSparkUI(appId, None) { ui => - new ApiStreamingRootResource(ui) - } + def getStreamingRoot(@PathParam("appId") appId: String): Class[ApiStreamingRootResource] = { + classOf[ApiStreamingRootResource] } @Path("applications/{appId}/{attemptId}/streaming") def getStreamingRoot( @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): ApiStreamingRootResource = { - withSparkUI(appId, Some(attemptId)) { ui => - new ApiStreamingRootResource(ui) + @PathParam("attemptId") attemptId: String): Class[ApiStreamingRootResource] = { + classOf[ApiStreamingRootResource] + } +} + +/** + * Base class for streaming API handlers, provides easy access to the streaming listener that + * holds the app's information. + */ +private[v1] trait BaseStreamingAppResource extends BaseAppResource { + + protected def withListener[T](fn: StreamingJobProgressListener => T): T = withUI { ui => + val listener = ui.getStreamingJobProgressListener match { + case Some(listener) => listener.asInstanceOf[StreamingJobProgressListener] + case None => throw new NotFoundException("no streaming listener attached to " + ui.getAppName) + } + listener.synchronized { + fn(listener) } } + } diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala index 1ccd586c848bd..a2571b910f615 100644 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala @@ -17,58 +17,180 @@ package org.apache.spark.status.api.v1.streaming -import javax.ws.rs.Path +import java.util.{Arrays => JArrays, Collections, Date, List => JList} +import javax.ws.rs.{GET, Path, PathParam, Produces, QueryParam} +import javax.ws.rs.core.MediaType import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.streaming.Time import org.apache.spark.streaming.ui.StreamingJobProgressListener +import org.apache.spark.streaming.ui.StreamingJobProgressListener._ import org.apache.spark.ui.SparkUI -private[v1] class ApiStreamingRootResource(ui: SparkUI) { - - import org.apache.spark.status.api.v1.streaming.ApiStreamingRootResource._ +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class ApiStreamingRootResource extends BaseStreamingAppResource { + @GET @Path("statistics") - def getStreamingStatistics(): StreamingStatisticsResource = { - new StreamingStatisticsResource(getListener(ui)) + def streamingStatistics(): StreamingStatistics = withListener { listener => + val batches = listener.retainedBatches + val avgInputRate = avgRate(batches.map(_.numRecords * 1000.0 / listener.batchDuration)) + val avgSchedulingDelay = avgTime(batches.flatMap(_.schedulingDelay)) + val avgProcessingTime = avgTime(batches.flatMap(_.processingDelay)) + val avgTotalDelay = avgTime(batches.flatMap(_.totalDelay)) + + new StreamingStatistics( + startTime = new Date(listener.startTime), + batchDuration = listener.batchDuration, + numReceivers = listener.numReceivers, + numActiveReceivers = listener.numActiveReceivers, + numInactiveReceivers = listener.numInactiveReceivers, + numTotalCompletedBatches = listener.numTotalCompletedBatches, + numRetainedCompletedBatches = listener.retainedCompletedBatches.size, + numActiveBatches = listener.numUnprocessedBatches, + numProcessedRecords = listener.numTotalProcessedRecords, + numReceivedRecords = listener.numTotalReceivedRecords, + avgInputRate = avgInputRate, + avgSchedulingDelay = avgSchedulingDelay, + avgProcessingTime = avgProcessingTime, + avgTotalDelay = avgTotalDelay + ) } + @GET @Path("receivers") - def getReceivers(): AllReceiversResource = { - new AllReceiversResource(getListener(ui)) + def receiversList(): Seq[ReceiverInfo] = withListener { listener => + listener.receivedRecordRateWithBatchTime.map { case (streamId, eventRates) => + val receiverInfo = listener.receiverInfo(streamId) + val streamName = receiverInfo.map(_.name) + .orElse(listener.streamName(streamId)).getOrElse(s"Stream-$streamId") + val avgEventRate = + if (eventRates.isEmpty) None else Some(eventRates.map(_._2).sum / eventRates.size) + + val (errorTime, errorMessage, error) = receiverInfo match { + case None => (None, None, None) + case Some(info) => + val someTime = + if (info.lastErrorTime >= 0) Some(new Date(info.lastErrorTime)) else None + val someMessage = + if (info.lastErrorMessage.length > 0) Some(info.lastErrorMessage) else None + val someError = + if (info.lastError.length > 0) Some(info.lastError) else None + + (someTime, someMessage, someError) + } + + new ReceiverInfo( + streamId = streamId, + streamName = streamName, + isActive = receiverInfo.map(_.active), + executorId = receiverInfo.map(_.executorId), + executorHost = receiverInfo.map(_.location), + lastErrorTime = errorTime, + lastErrorMessage = errorMessage, + lastError = error, + avgEventRate = avgEventRate, + eventRates = eventRates + ) + }.toSeq.sortBy(_.streamId) } + @GET @Path("receivers/{streamId: \\d+}") - def getReceiver(): OneReceiverResource = { - new OneReceiverResource(getListener(ui)) + def oneReceiver(@PathParam("streamId") streamId: Int): ReceiverInfo = { + receiversList().find { _.streamId == streamId }.getOrElse( + throw new NotFoundException("unknown receiver: " + streamId)) } + @GET @Path("batches") - def getBatches(): AllBatchesResource = { - new AllBatchesResource(getListener(ui)) + def batchesList(@QueryParam("status") statusParams: JList[BatchStatus]): Seq[BatchInfo] = { + withListener { listener => + val statuses = + if (statusParams.isEmpty) JArrays.asList(BatchStatus.values(): _*) else statusParams + val statusToBatches = Seq( + BatchStatus.COMPLETED -> listener.retainedCompletedBatches, + BatchStatus.QUEUED -> listener.waitingBatches, + BatchStatus.PROCESSING -> listener.runningBatches + ) + + val batchInfos = for { + (status, batches) <- statusToBatches + batch <- batches if statuses.contains(status) + } yield { + val batchId = batch.batchTime.milliseconds + val firstFailureReason = batch.outputOperations.flatMap(_._2.failureReason).headOption + + new BatchInfo( + batchId = batchId, + batchTime = new Date(batchId), + status = status.toString, + batchDuration = listener.batchDuration, + inputSize = batch.numRecords, + schedulingDelay = batch.schedulingDelay, + processingTime = batch.processingDelay, + totalDelay = batch.totalDelay, + numActiveOutputOps = batch.numActiveOutputOp, + numCompletedOutputOps = batch.numCompletedOutputOp, + numFailedOutputOps = batch.numFailedOutputOp, + numTotalOutputOps = batch.outputOperations.size, + firstFailureReason = firstFailureReason + ) + } + + batchInfos.sortBy(- _.batchId) + } } + @GET @Path("batches/{batchId: \\d+}") - def getBatch(): OneBatchResource = { - new OneBatchResource(getListener(ui)) + def oneBatch(@PathParam("batchId") batchId: Long): BatchInfo = { + batchesList(Collections.emptyList()).find { _.batchId == batchId }.getOrElse( + throw new NotFoundException("unknown batch: " + batchId)) } + @GET @Path("batches/{batchId: \\d+}/operations") - def getOutputOperations(): AllOutputOperationsResource = { - new AllOutputOperationsResource(getListener(ui)) + def operationsList(@PathParam("batchId") batchId: Long): Seq[OutputOperationInfo] = { + withListener { listener => + val ops = listener.getBatchUIData(Time(batchId)) match { + case Some(batch) => + for ((opId, op) <- batch.outputOperations) yield { + val jobIds = batch.outputOpIdSparkJobIdPairs + .filter(_.outputOpId == opId).map(_.sparkJobId).toSeq.sorted + + new OutputOperationInfo( + outputOpId = opId, + name = op.name, + description = op.description, + startTime = op.startTime.map(new Date(_)), + endTime = op.endTime.map(new Date(_)), + duration = op.duration, + failureReason = op.failureReason, + jobIds = jobIds + ) + } + case None => throw new NotFoundException("unknown batch: " + batchId) + } + ops.toSeq + } } + @GET @Path("batches/{batchId: \\d+}/operations/{outputOpId: \\d+}") - def getOutputOperation(): OneOutputOperationResource = { - new OneOutputOperationResource(getListener(ui)) + def oneOperation( + @PathParam("batchId") batchId: Long, + @PathParam("outputOpId") opId: OutputOpId): OutputOperationInfo = { + operationsList(batchId).find { _.outputOpId == opId }.getOrElse( + throw new NotFoundException("unknown output operation: " + opId)) } -} + private def avgRate(data: Seq[Double]): Option[Double] = { + if (data.isEmpty) None else Some(data.sum / data.size) + } -private[v1] object ApiStreamingRootResource { - def getListener(ui: SparkUI): StreamingJobProgressListener = { - ui.getStreamingJobProgressListener match { - case Some(listener) => listener.asInstanceOf[StreamingJobProgressListener] - case None => throw new NotFoundException("no streaming listener attached to " + ui.getAppName) - } + private def avgTime(data: Seq[Long]): Option[Long] = { + if (data.isEmpty) None else Some(data.sum / data.size) } + } diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala deleted file mode 100644 index aabcdb29b0d4c..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala +++ /dev/null @@ -1,39 +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.status.api.v1.streaming - -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.status.api.v1.NotFoundException -import org.apache.spark.streaming.ui.StreamingJobProgressListener -import org.apache.spark.streaming.ui.StreamingJobProgressListener._ - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneOutputOperationResource(listener: StreamingJobProgressListener) { - - @GET - def oneOperation( - @PathParam("batchId") batchId: Long, - @PathParam("outputOpId") opId: OutputOpId): OutputOperationInfo = { - - val someOutputOp = AllOutputOperationsResource.outputOperationInfoList(listener, batchId) - .find { _.outputOpId == opId } - someOutputOp.getOrElse(throw new NotFoundException("unknown output operation: " + opId)) - } -} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.scala deleted file mode 100644 index c0cc99da3a9c7..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.scala +++ /dev/null @@ -1,35 +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.status.api.v1.streaming - -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.status.api.v1.NotFoundException -import org.apache.spark.streaming.ui.StreamingJobProgressListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneReceiverResource(listener: StreamingJobProgressListener) { - - @GET - def oneReceiver(@PathParam("streamId") streamId: Int): ReceiverInfo = { - val someReceiver = AllReceiversResource.receiverInfoList(listener) - .find { _.streamId == streamId } - someReceiver.getOrElse(throw new NotFoundException("unknown receiver: " + streamId)) - } -} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.scala deleted file mode 100644 index 6cff87be59ca8..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.scala +++ /dev/null @@ -1,64 +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.status.api.v1.streaming - -import java.util.Date -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.streaming.ui.StreamingJobProgressListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class StreamingStatisticsResource(listener: StreamingJobProgressListener) { - - @GET - def streamingStatistics(): StreamingStatistics = { - listener.synchronized { - val batches = listener.retainedBatches - val avgInputRate = avgRate(batches.map(_.numRecords * 1000.0 / listener.batchDuration)) - val avgSchedulingDelay = avgTime(batches.flatMap(_.schedulingDelay)) - val avgProcessingTime = avgTime(batches.flatMap(_.processingDelay)) - val avgTotalDelay = avgTime(batches.flatMap(_.totalDelay)) - - new StreamingStatistics( - startTime = new Date(listener.startTime), - batchDuration = listener.batchDuration, - numReceivers = listener.numReceivers, - numActiveReceivers = listener.numActiveReceivers, - numInactiveReceivers = listener.numInactiveReceivers, - numTotalCompletedBatches = listener.numTotalCompletedBatches, - numRetainedCompletedBatches = listener.retainedCompletedBatches.size, - numActiveBatches = listener.numUnprocessedBatches, - numProcessedRecords = listener.numTotalProcessedRecords, - numReceivedRecords = listener.numTotalReceivedRecords, - avgInputRate = avgInputRate, - avgSchedulingDelay = avgSchedulingDelay, - avgProcessingTime = avgProcessingTime, - avgTotalDelay = avgTotalDelay - ) - } - } - - private def avgRate(data: Seq[Double]): Option[Double] = { - if (data.isEmpty) None else Some(data.sum / data.size) - } - - private def avgTime(data: Seq[Long]): Option[Long] = { - if (data.isEmpty) None else Some(data.sum / data.size) - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index f55af6a5cc358..a25878c4028dc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -23,16 +23,16 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.status.api.v1.{JobData, StageData} import org.apache.spark.streaming.Time import org.apache.spark.streaming.ui.StreamingJobProgressListener.SparkJobId import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} -import org.apache.spark.ui.jobs.UIData.JobUIData -private[ui] case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) +private[ui] case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobData: Option[JobData]) private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private val streamingListener = parent.listener - private val sparkListener = parent.ssc.sc.jobProgressListener + private val store = parent.parent.store private def columns: Seq[Node] = { Output Op Id @@ -52,13 +52,13 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { formattedOutputOpDuration: String, numSparkJobRowsInOutputOp: Int, isFirstRow: Boolean, - sparkJob: SparkJobIdWithUIData): Seq[Node] = { - if (sparkJob.jobUIData.isDefined) { + jobIdWithData: SparkJobIdWithUIData): Seq[Node] = { + if (jobIdWithData.jobData.isDefined) { generateNormalJobRow(outputOpData, outputOpDescription, formattedOutputOpDuration, - numSparkJobRowsInOutputOp, isFirstRow, sparkJob.jobUIData.get) + numSparkJobRowsInOutputOp, isFirstRow, jobIdWithData.jobData.get) } else { generateDroppedJobRow(outputOpData, outputOpDescription, formattedOutputOpDuration, - numSparkJobRowsInOutputOp, isFirstRow, sparkJob.sparkJobId) + numSparkJobRowsInOutputOp, isFirstRow, jobIdWithData.sparkJobId) } } @@ -94,15 +94,15 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { formattedOutputOpDuration: String, numSparkJobRowsInOutputOp: Int, isFirstRow: Boolean, - sparkJob: JobUIData): Seq[Node] = { + sparkJob: JobData): Seq[Node] = { val duration: Option[Long] = { sparkJob.submissionTime.map { start => - val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) - end - start + val end = sparkJob.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) + end - start.getTime() } } val lastFailureReason = - sparkJob.stageIds.sorted.reverse.flatMap(sparkListener.stageIdToInfo.get). + sparkJob.stageIds.sorted.reverse.flatMap(getStageData). dropWhile(_.failureReason == None).take(1). // get the first info that contains failure flatMap(info => info.failureReason).headOption.getOrElse("") val formattedDuration = duration.map(d => SparkUIUtils.formatDuration(d)).getOrElse("-") @@ -135,7 +135,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { {formattedDuration} - {sparkJob.completedStageIndices.size}/{sparkJob.stageIds.size - sparkJob.numSkippedStages} + {sparkJob.numCompletedStages}/{sparkJob.stageIds.size - sparkJob.numSkippedStages} {if (sparkJob.numFailedStages > 0) s"(${sparkJob.numFailedStages} failed)"} {if (sparkJob.numSkippedStages > 0) s"(${sparkJob.numSkippedStages} skipped)"} @@ -146,7 +146,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { completed = sparkJob.numCompletedTasks, failed = sparkJob.numFailedTasks, skipped = sparkJob.numSkippedTasks, - reasonToNumKilled = sparkJob.reasonToNumKilled, + reasonToNumKilled = sparkJob.killedTasksSummary, total = sparkJob.numTasks - sparkJob.numSkippedTasks) } @@ -246,11 +246,19 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") {
} - private def getJobData(sparkJobId: SparkJobId): Option[JobUIData] = { - sparkListener.activeJobs.get(sparkJobId).orElse { - sparkListener.completedJobs.find(_.jobId == sparkJobId).orElse { - sparkListener.failedJobs.find(_.jobId == sparkJobId) - } + private def getJobData(sparkJobId: SparkJobId): Option[JobData] = { + try { + Some(store.job(sparkJobId)) + } catch { + case _: NoSuchElementException => None + } + } + + private def getStageData(stageId: Int): Option[StageData] = { + try { + Some(store.lastStageAttempt(stageId)) + } catch { + case _: NoSuchElementException => None } } @@ -282,25 +290,22 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { val sparkJobIds = outputOpIdToSparkJobIds.getOrElse(outputOpId, Seq.empty) (outputOperation, sparkJobIds) }.toSeq.sortBy(_._1.id) - sparkListener.synchronized { - val outputOpWithJobs = outputOps.map { case (outputOpData, sparkJobIds) => - (outputOpData, - sparkJobIds.map(sparkJobId => SparkJobIdWithUIData(sparkJobId, getJobData(sparkJobId)))) - } + val outputOpWithJobs = outputOps.map { case (outputOpData, sparkJobIds) => + (outputOpData, sparkJobIds.map { jobId => SparkJobIdWithUIData(jobId, getJobData(jobId)) }) + } - - - {columns} - - - { - outputOpWithJobs.map { case (outputOpData, sparkJobIds) => - generateOutputOpIdRow(outputOpData, sparkJobIds) - } +
+ + {columns} + + + { + outputOpWithJobs.map { case (outputOpData, sparkJobs) => + generateOutputOpIdRow(outputOpData, sparkJobs) } - -
- } + } + + } def render(request: HttpServletRequest): Seq[Node] = streamingListener.synchronized {