@@ -132,7 +133,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val totalTasksNumStr = if (totalTasks == storedTasks) {
s"$totalTasks"
} else {
- s"$storedTasks, showing ${totalTasks}"
+ s"$totalTasks, showing $storedTasks"
}
val summary =
@@ -685,7 +686,7 @@ private[ui] class TaskDataSource(
private var _tasksToShow: Seq[TaskData] = null
- override def dataSize: Int = taskCount(stage)
+ override def dataSize: Int = store.taskCount(stage.stageId, stage.attemptId).toInt
override def sliceData(from: Int, to: Int): Seq[TaskData] = {
if (_tasksToShow == null) {
@@ -1051,9 +1052,4 @@ private[ui] object ApiHelper {
(stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name))
}
- def taskCount(stageData: StageData): Int = {
- stageData.numActiveTasks + stageData.numCompleteTasks + stageData.numFailedTasks +
- stageData.numKilledTasks
- }
-
}
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index b6c300c4778b1..6c4740c002103 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -175,7 +175,7 @@ private[spark] object ClosureCleaner extends Logging {
closure.getClass.isSynthetic &&
closure
.getClass
- .getInterfaces.exists(_.getName.equals("scala.Serializable"))
+ .getInterfaces.exists(_.getName == "scala.Serializable")
if (isClosureCandidate) {
try {
@@ -285,8 +285,6 @@ private[spark] object ClosureCleaner extends Logging {
innerClasses.foreach { c => logDebug(s" ${c.getName}") }
logDebug(s" + outer classes: ${outerClasses.size}" )
outerClasses.foreach { c => logDebug(s" ${c.getName}") }
- logDebug(s" + outer objects: ${outerObjects.size}")
- outerObjects.foreach { o => logDebug(s" $o") }
}
// Fail fast if we detect return statements in closures
@@ -318,19 +316,20 @@ private[spark] object ClosureCleaner extends Logging {
if (outerPairs.nonEmpty) {
val (outermostClass, outermostObject) = outerPairs.head
if (isClosure(outermostClass)) {
- logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}")
+ logDebug(s" + outermost object is a closure, so we clone it: ${outermostClass}")
} else if (outermostClass.getName.startsWith("$line")) {
// SPARK-14558: if the outermost object is a REPL line object, we should clone
// and clean it as it may carray a lot of unnecessary information,
// e.g. hadoop conf, spark conf, etc.
- logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}")
+ logDebug(s" + outermost object is a REPL line object, so we clone it:" +
+ s" ${outermostClass}")
} else {
// The closure is ultimately nested inside a class; keep the object of that
// class without cloning it since we don't want to clone the user's objects.
// Note that we still need to keep around the outermost object itself because
// we need it to clone its child closure later (see below).
- logDebug(" + outermost object is not a closure or REPL line object," +
- "so do not clone it: " + outerPairs.head)
+ logDebug(s" + outermost object is not a closure or REPL line object," +
+ s" so do not clone it: ${outermostClass}")
parent = outermostObject // e.g. SparkContext
outerPairs = outerPairs.tail
}
@@ -341,7 +340,7 @@ private[spark] object ClosureCleaner extends Logging {
// Clone the closure objects themselves, nulling out any fields that are not
// used in the closure we're working on or any of its inner closures.
for ((cls, obj) <- outerPairs) {
- logDebug(s" + cloning the object $obj of class ${cls.getName}")
+ logDebug(s" + cloning instance of class ${cls.getName}")
// We null out these unused references by cloning each object and then filling in all
// required fields from the original object. We need the parent here because the Java
// language specification requires the first constructor parameter of any closure to be
@@ -351,7 +350,7 @@ private[spark] object ClosureCleaner extends Logging {
// If transitive cleaning is enabled, we recursively clean any enclosing closure using
// the already populated accessed fields map of the starting closure
if (cleanTransitively && isClosure(clone.getClass)) {
- logDebug(s" + cleaning cloned closure $clone recursively (${cls.getName})")
+ logDebug(s" + cleaning cloned closure recursively (${cls.getName})")
// No need to check serializable here for the outer closures because we're
// only interested in the serializability of the starting closure
clean(clone, checkSerializable = false, cleanTransitively, accessedFields)
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 50c6461373dee..0cd8612b8fd1c 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -31,6 +31,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark._
import org.apache.spark.executor._
+import org.apache.spark.metrics.ExecutorMetricType
import org.apache.spark.rdd.RDDOperationScope
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.ExecutorInfo
@@ -98,6 +99,8 @@ private[spark] object JsonProtocol {
logStartToJson(logStart)
case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
executorMetricsUpdateToJson(metricsUpdate)
+ case stageExecutorMetrics: SparkListenerStageExecutorMetrics =>
+ stageExecutorMetricsToJson(stageExecutorMetrics)
case blockUpdate: SparkListenerBlockUpdated =>
blockUpdateToJson(blockUpdate)
case _ => parse(mapper.writeValueAsString(event))
@@ -236,6 +239,7 @@ private[spark] object JsonProtocol {
def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = {
val execId = metricsUpdate.execId
val accumUpdates = metricsUpdate.accumUpdates
+ val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_))
("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~
("Executor ID" -> execId) ~
("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) =>
@@ -243,7 +247,16 @@ private[spark] object JsonProtocol {
("Stage ID" -> stageId) ~
("Stage Attempt ID" -> stageAttemptId) ~
("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList))
- })
+ }) ~
+ ("Executor Metrics Updated" -> executorMetrics)
+ }
+
+ def stageExecutorMetricsToJson(metrics: SparkListenerStageExecutorMetrics): JValue = {
+ ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageExecutorMetrics) ~
+ ("Executor ID" -> metrics.execId) ~
+ ("Stage ID" -> metrics.stageId) ~
+ ("Stage Attempt ID" -> metrics.stageAttemptId) ~
+ ("Executor Metrics" -> executorMetricsToJson(metrics.executorMetrics))
}
def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = {
@@ -379,6 +392,14 @@ private[spark] object JsonProtocol {
("Updated Blocks" -> updatedBlocks)
}
+ /** Convert executor metrics to JSON. */
+ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = {
+ val metrics = ExecutorMetricType.values.map{ metricType =>
+ JField(metricType.name, executorMetrics.getMetricValue(metricType))
+ }
+ JObject(metrics: _*)
+ }
+
def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = {
val reason = Utils.getFormattedClassName(taskEndReason)
val json: JObject = taskEndReason match {
@@ -531,6 +552,7 @@ private[spark] object JsonProtocol {
val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved)
val logStart = Utils.getFormattedClassName(SparkListenerLogStart)
val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate)
+ val stageExecutorMetrics = Utils.getFormattedClassName(SparkListenerStageExecutorMetrics)
val blockUpdate = Utils.getFormattedClassName(SparkListenerBlockUpdated)
}
@@ -555,6 +577,7 @@ private[spark] object JsonProtocol {
case `executorRemoved` => executorRemovedFromJson(json)
case `logStart` => logStartFromJson(json)
case `metricsUpdate` => executorMetricsUpdateFromJson(json)
+ case `stageExecutorMetrics` => stageExecutorMetricsFromJson(json)
case `blockUpdate` => blockUpdateFromJson(json)
case other => mapper.readValue(compact(render(json)), Utils.classForName(other))
.asInstanceOf[SparkListenerEvent]
@@ -585,6 +608,15 @@ private[spark] object JsonProtocol {
SparkListenerTaskGettingResult(taskInfo)
}
+ /** Extract the executor metrics from JSON. */
+ def executorMetricsFromJson(json: JValue): ExecutorMetrics = {
+ val metrics =
+ ExecutorMetricType.values.map { metric =>
+ metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L)
+ }.toMap
+ new ExecutorMetrics(metrics)
+ }
+
def taskEndFromJson(json: JValue): SparkListenerTaskEnd = {
val stageId = (json \ "Stage ID").extract[Int]
val stageAttemptId =
@@ -691,7 +723,18 @@ private[spark] object JsonProtocol {
(json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson)
(taskId, stageId, stageAttemptId, updates)
}
- SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates)
+ val executorUpdates = jsonOption(json \ "Executor Metrics Updated").map {
+ executorUpdate => executorMetricsFromJson(executorUpdate)
+ }
+ SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates)
+ }
+
+ def stageExecutorMetricsFromJson(json: JValue): SparkListenerStageExecutorMetrics = {
+ val execId = (json \ "Executor ID").extract[String]
+ val stageId = (json \ "Stage ID").extract[Int]
+ val stageAttemptId = (json \ "Stage Attempt ID").extract[Int]
+ val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics")
+ SparkListenerStageExecutorMetrics(execId, stageId, stageAttemptId, executorMetrics)
}
def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = {
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 15c958d3f511e..93b5826f8a74b 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -19,7 +19,6 @@ package org.apache.spark.util
import java.io._
import java.lang.{Byte => JByte}
-import java.lang.InternalError
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
@@ -240,6 +239,19 @@ private[spark] object Utils extends Logging {
// scalastyle:on classforname
}
+ /**
+ * Run a segment of code using a different context class loader in the current thread
+ */
+ def withContextClassLoader[T](ctxClassLoader: ClassLoader)(fn: => T): T = {
+ val oldClassLoader = Thread.currentThread().getContextClassLoader()
+ try {
+ Thread.currentThread().setContextClassLoader(ctxClassLoader)
+ fn
+ } finally {
+ Thread.currentThread().setContextClassLoader(oldClassLoader)
+ }
+ }
+
/**
* Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]]
*/
@@ -2052,6 +2064,30 @@ private[spark] object Utils extends Logging {
}
}
+ /**
+ * Implements the same logic as JDK `java.lang.String#trim` by removing leading and trailing
+ * non-printable characters less or equal to '\u0020' (SPACE) but preserves natural line
+ * delimiters according to [[java.util.Properties]] load method. The natural line delimiters are
+ * removed by JDK during load. Therefore any remaining ones have been specifically provided and
+ * escaped by the user, and must not be ignored
+ *
+ * @param str
+ * @return the trimmed value of str
+ */
+ private[util] def trimExceptCRLF(str: String): String = {
+ val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch =>
+ ch > ' ' || ch == '\r' || ch == '\n'
+ }
+
+ val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter)
+ val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter)
+ if (firstPos >= 0 && lastPos >= 0) {
+ str.substring(firstPos, lastPos + 1)
+ } else {
+ ""
+ }
+ }
+
/** Load properties present in the given file. */
def getPropertiesFromFile(filename: String): Map[String, String] = {
val file = new File(filename)
@@ -2062,8 +2098,10 @@ private[spark] object Utils extends Logging {
try {
val properties = new Properties()
properties.load(inReader)
- properties.stringPropertyNames().asScala.map(
- k => (k, properties.getProperty(k).trim)).toMap
+ properties.stringPropertyNames().asScala
+ .map { k => (k, trimExceptCRLF(properties.getProperty(k))) }
+ .toMap
+
} catch {
case e: IOException =>
throw new SparkException(s"Failed when loading Spark properties from $filename", e)
@@ -2698,7 +2736,7 @@ private[spark] object Utils extends Logging {
}
val masterScheme = new URI(masterWithoutK8sPrefix).getScheme
- val resolvedURL = masterScheme.toLowerCase match {
+ val resolvedURL = masterScheme.toLowerCase(Locale.ROOT) match {
case "https" =>
masterWithoutK8sPrefix
case "http" =>
@@ -2795,6 +2833,36 @@ private[spark] object Utils extends Logging {
}
}
}
+
+ /**
+ * Regular expression matching full width characters.
+ *
+ * Looked at all the 0x0000-0xFFFF characters (unicode) and showed them under Xshell.
+ * Found all the full width characters, then get the regular expression.
+ */
+ private val fullWidthRegex = ("""[""" +
+ // scalastyle:off nonascii
+ """\u1100-\u115F""" +
+ """\u2E80-\uA4CF""" +
+ """\uAC00-\uD7A3""" +
+ """\uF900-\uFAFF""" +
+ """\uFE10-\uFE19""" +
+ """\uFE30-\uFE6F""" +
+ """\uFF00-\uFF60""" +
+ """\uFFE0-\uFFE6""" +
+ // scalastyle:on nonascii
+ """]""").r
+
+ /**
+ * Return the number of half widths in a given string. Note that a full width character
+ * occupies two half widths.
+ *
+ * For a string consisting of 1 million characters, the execution of this method requires
+ * about 50ms.
+ */
+ def stringHalfWidth(str: String): Int = {
+ if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size
+ }
}
private[util] object CallerContext extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index 39f050f6ca5ad..4aa8d45ec7404 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -19,17 +19,16 @@ package org.apache.spark.util.io
import java.io.{File, FileInputStream, InputStream}
import java.nio.ByteBuffer
-import java.nio.channels.{FileChannel, WritableByteChannel}
-import java.nio.file.StandardOpenOption
-
-import scala.collection.mutable.ListBuffer
+import java.nio.channels.WritableByteChannel
+import com.google.common.io.ByteStreams
import com.google.common.primitives.UnsignedBytes
+import org.apache.commons.io.IOUtils
import org.apache.spark.SparkEnv
import org.apache.spark.internal.config
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
-import org.apache.spark.network.util.ByteArrayWritableChannel
+import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream}
import org.apache.spark.storage.StorageUtils
import org.apache.spark.util.Utils
@@ -175,30 +174,36 @@ object ChunkedByteBuffer {
def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = {
data match {
case f: FileSegmentManagedBuffer =>
- map(f.getFile, maxChunkSize, f.getOffset, f.getLength)
+ fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength)
case other =>
new ChunkedByteBuffer(other.nioByteBuffer())
}
}
- def map(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
- map(file, maxChunkSize, 0, file.length())
+ def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
+ fromFile(file, maxChunkSize, 0, file.length())
}
- def map(file: File, maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = {
- Utils.tryWithResource(FileChannel.open(file.toPath, StandardOpenOption.READ)) { channel =>
- var remaining = length
- var pos = offset
- val chunks = new ListBuffer[ByteBuffer]()
- while (remaining > 0) {
- val chunkSize = math.min(remaining, maxChunkSize)
- val chunk = channel.map(FileChannel.MapMode.READ_ONLY, pos, chunkSize)
- pos += chunkSize
- remaining -= chunkSize
- chunks += chunk
- }
- new ChunkedByteBuffer(chunks.toArray)
+ private def fromFile(
+ file: File,
+ maxChunkSize: Int,
+ offset: Long,
+ length: Long): ChunkedByteBuffer = {
+ // We do *not* memory map the file, because we may end up putting this into the memory store,
+ // and spark currently is not expecting memory-mapped buffers in the memory store, it conflicts
+ // with other parts that manage the lifecyle of buffers and dispose them. See SPARK-25422.
+ val is = new FileInputStream(file)
+ ByteStreams.skipFully(is, offset)
+ val in = new LimitedInputStream(is, length)
+ val chunkSize = math.min(maxChunkSize, length).toInt
+ val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _)
+ Utils.tryWithSafeFinally {
+ IOUtils.copy(in, out)
+ } {
+ in.close()
+ out.close()
}
+ out.toChunkedByteBuffer
}
}
diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java
new file mode 100644
index 0000000000000..686eb28010c6a
--- /dev/null
+++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java
@@ -0,0 +1,139 @@
+/*
+ * 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;
+
+import org.apache.spark.api.java.JavaSparkContext;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class ExecutorPluginSuite {
+ private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins";
+ private static final String testBadPluginName = TestBadShutdownPlugin.class.getName();
+ private static final String testPluginName = TestExecutorPlugin.class.getName();
+ private static final String testSecondPluginName = TestSecondPlugin.class.getName();
+
+ // Static value modified by testing plugins to ensure plugins loaded correctly.
+ public static int numSuccessfulPlugins = 0;
+
+ // Static value modified by testing plugins to verify plugins shut down properly.
+ public static int numSuccessfulTerminations = 0;
+
+ private JavaSparkContext sc;
+
+ @Before
+ public void setUp() {
+ sc = null;
+ numSuccessfulPlugins = 0;
+ numSuccessfulTerminations = 0;
+ }
+
+ @After
+ public void tearDown() {
+ if (sc != null) {
+ sc.stop();
+ sc = null;
+ }
+ }
+
+ private SparkConf initializeSparkConf(String pluginNames) {
+ return new SparkConf()
+ .setMaster("local")
+ .setAppName("test")
+ .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames);
+ }
+
+ @Test
+ public void testPluginClassDoesNotExist() {
+ SparkConf conf = initializeSparkConf("nonexistant.plugin");
+ try {
+ sc = new JavaSparkContext(conf);
+ fail("No exception thrown for nonexistant plugin");
+ } catch (Exception e) {
+ // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown
+ assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException"));
+ }
+ }
+
+ @Test
+ public void testAddPlugin() throws InterruptedException {
+ // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins
+ SparkConf conf = initializeSparkConf(testPluginName);
+ sc = new JavaSparkContext(conf);
+ assertEquals(1, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(1, numSuccessfulTerminations);
+ }
+
+ @Test
+ public void testAddMultiplePlugins() throws InterruptedException {
+ // Load two plugins and verify they both execute.
+ SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName);
+ sc = new JavaSparkContext(conf);
+ assertEquals(2, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(2, numSuccessfulTerminations);
+ }
+
+ @Test
+ public void testPluginShutdownWithException() {
+ // Verify an exception in one plugin shutdown does not affect the others
+ String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName;
+ SparkConf conf = initializeSparkConf(pluginNames);
+ sc = new JavaSparkContext(conf);
+ assertEquals(3, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(2, numSuccessfulTerminations);
+ }
+
+ public static class TestExecutorPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ ExecutorPluginSuite.numSuccessfulTerminations++;
+ }
+ }
+
+ public static class TestSecondPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ ExecutorPluginSuite.numSuccessfulTerminations++;
+ }
+ }
+
+ public static class TestBadShutdownPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ throw new RuntimeException("This plugin will fail to cleanly shut down");
+ }
+ }
+}
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index d7d2d0b012bd3..a0664b30d6cc2 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -76,7 +76,7 @@ public void freeingPageSetsPageNumberToSpecialConstant() {
final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
final MemoryBlock dataPage = manager.allocatePage(256, c);
c.freePage(dataPage);
- Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.getPageNumber());
+ Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber);
}
@Test(expected = AssertionError.class)
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index faa70f23b0ac6..0d5c5ea7903e9 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -233,7 +233,6 @@ public void writeEmptyIterator() throws Exception {
writer.write(Iterators.emptyIterator());
final Option
mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
- assertEquals(0, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile);
assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten());
@@ -253,7 +252,6 @@ public void writeWithoutSpilling() throws Exception {
writer.write(dataToWrite.iterator());
final Option mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
- assertEquals(NUM_PARTITITONS, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
long sumOfPartitionSizes = 0;
diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
index 4fecf84db65a2..eea6f595efd2a 100644
--- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
index 4fecf84db65a2..7bc7f31be097b 100644
--- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134,
+ "lastUpdatedEpoch" : 0
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
new file mode 100644
index 0000000000000..9bf2086cc8e72
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
@@ -0,0 +1,314 @@
+[ {
+ "id" : "driver",
+ "hostPort" : "node0033.grid.company.com:60749",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 0,
+ "maxTasks" : 0,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 1043437977,
+ "addTime" : "2018-04-19T23:55:05.107GMT",
+ "executorLogs" : { },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 1043437977,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 905801,
+ "JVMOffHeapMemory" : 205304696,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 905801,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 397602,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 629553808,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "7",
+ "hostPort" : "node6340.grid.company.com:5933",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:49.826GMT",
+ "executorLogs" : {
+ "stdout" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096",
+ "stderr" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "6",
+ "hostPort" : "node6644.grid.company.com:8445",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:47.549GMT",
+ "executorLogs" : {
+ "stdout" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096",
+ "stderr" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "5",
+ "hostPort" : "node2477.grid.company.com:20123",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 9252,
+ "totalGCTime" : 920,
+ "totalInputBytes" : 36838295,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 355051,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:43.160GMT",
+ "executorLogs" : {
+ "stdout" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096",
+ "stderr" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "4",
+ "hostPort" : "node4243.grid.company.com:16084",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 3,
+ "totalTasks" : 3,
+ "totalDuration" : 15645,
+ "totalGCTime" : 405,
+ "totalInputBytes" : 87272855,
+ "totalShuffleRead" : 438675,
+ "totalShuffleWrite" : 26773039,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.278GMT",
+ "executorLogs" : {
+ "stdout" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096",
+ "stderr" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 63104457,
+ "JVMOffHeapMemory" : 95657456,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 100853193,
+ "OnHeapExecutionMemory" : 37748736,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 126261,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 518613056,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "3",
+ "hostPort" : "node0998.grid.company.com:45265",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 14491,
+ "totalGCTime" : 342,
+ "totalInputBytes" : 50409514,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 31362123,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.088GMT",
+ "executorLogs" : {
+ "stdout" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096",
+ "stderr" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 69535048,
+ "JVMOffHeapMemory" : 90709624,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 69535048,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 87796,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 726805712,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "2",
+ "hostPort" : "node4045.grid.company.com:29262",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 14113,
+ "totalGCTime" : 326,
+ "totalInputBytes" : 50423423,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 22950296,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.471GMT",
+ "executorLogs" : {
+ "stdout" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096",
+ "stderr" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 58468944,
+ "JVMOffHeapMemory" : 91208368,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 58468944,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 87796,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 595946552,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "1",
+ "hostPort" : "node1404.grid.company.com:34043",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 3,
+ "totalTasks" : 3,
+ "totalDuration" : 15665,
+ "totalGCTime" : 471,
+ "totalInputBytes" : 98905018,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 20594744,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:11.695GMT",
+ "executorLogs" : {
+ "stdout" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096",
+ "stderr" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 47962185,
+ "JVMOffHeapMemory" : 100519936,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 47962185,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 98230,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 755008624,
+ "OffHeapStorageMemory" : 0
+ }
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
index 79950b0dc6486..9e1e65a358815 100644
--- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
@@ -28,19 +43,4 @@
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
-}, {
- "id" : "app-20161116163331-0000",
- "name" : "Spark shell",
- "attempts" : [ {
- "startTime" : "2016-11-16T22:33:29.916GMT",
- "endTime" : "2016-11-16T22:33:40.587GMT",
- "lastUpdated" : "",
- "duration" : 10671,
- "sparkUser" : "jose",
- "completed" : true,
- "appSparkVersion" : "2.1.0-SNAPSHOT",
- "lastUpdatedEpoch" : 0,
- "startTimeEpoch" : 1479335609916,
- "endTimeEpoch" : 1479335620587
- } ]
} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
index 7d60977dcd4fe..28c6bf1b3e01e 100644
--- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
index dfbfd8aedcc23..f547b79f47e1a 100644
--- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
@@ -101,4 +116,4 @@
"startTimeEpoch" : 1430917380880,
"endTimeEpoch" : 1430917380890
} ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151
new file mode 100644
index 0000000000000..c48ed741c56e0
--- /dev/null
+++ b/core/src/test/resources/spark-events/application_1506645932520_24630151
@@ -0,0 +1,63 @@
+{"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node0033.grid.company.com","Port":60749},"Maximum Memory":1043437977,"Timestamp":1524182105107,"Maximum Onheap Memory":1043437977,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"node0033.grid.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"57705","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://node0033.grid.company.com:57705/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.grid.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-21b68b4b-c1db-460e-a228-b87545d870f1/repl-58778a76-04c1-434d-bfb7-9a9b83afe718","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.grid.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.search.packages":"com.company.dali:dali-data-spark,com.company.spark-common:spark-common","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.default.packages":"com.company.dali:dali-data-spark:8.+?classifier=all,com.company.spark-common:spark-common_2.10:0.+?","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.grid.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://node0033.grid.company.com:8364","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.grid.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.grid.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"UTF-8","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1524182082734,"User":"edlu"}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182111695,"Executor ID":"1","Executor Info":{"Host":"node1404.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096","stderr":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node1404.grid.company.com","Port":34043},"Maximum Memory":956615884,"Timestamp":1524182111795,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112088,"Executor ID":"3","Executor Info":{"Host":"node0998.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096","stderr":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"node0998.grid.company.com","Port":45265},"Maximum Memory":956615884,"Timestamp":1524182112208,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112278,"Executor ID":"4","Executor Info":{"Host":"node4243.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096","stderr":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"node4243.grid.company.com","Port":16084},"Maximum Memory":956615884,"Timestamp":1524182112408,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112471,"Executor ID":"2","Executor Info":{"Host":"node4045.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096","stderr":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4045.grid.company.com","Port":29262},"Maximum Memory":956615884,"Timestamp":1524182112578,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182125829}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1524182125832}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182128463}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1524182128463}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1524182129952}
+{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1524182130194,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.