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:
*
*
- *
- *
serialization: this feature is not optional; 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.
+ *
Serialization
*
- *
key management: by using {@link #read(Class, Object)} and {@link #write(Class, Object)},
- * applications can leave key management to the implementation. For applications that want to
- * manage their own keys, the {@link #get(byte[], Class)} and {@link #set(byte[], Object)} methods
- * are available.
- *
+ *
+ * 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.sparkspark-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 =
}
+
+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 =
-
}
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 =
.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 =
-
++
+ 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 ++=