From 11c024165e7d0ccc5b5c9f23e9406f8921528d2b Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 7 Dec 2020 13:51:17 +0900 Subject: [PATCH 1/3] Revert "[SPARK-31953][SS] Add Spark Structured Streaming History Server Support" This reverts commit 4f9667035886a67e6c9a4e8fad2efa390e87ca68. --- dev/.rat-excludes | 1 - ...apache.spark.status.AppHistoryServerPlugin | 1 - .../streaming/StreamingQueryListenerBus.scala | 26 +-- .../StreamingQueryHistoryServerPlugin.scala | 43 ----- .../ui/StreamingQueryStatusStore.scala | 53 ------ .../spark/sql/internal/SharedState.scala | 8 +- .../sql/streaming/StreamingQueryManager.scala | 3 +- .../sql/streaming/ui/StreamingQueryPage.scala | 44 ++--- .../ui/StreamingQueryStatisticsPage.scala | 27 ++- .../ui/StreamingQueryStatusListener.scala | 166 +++++++----------- .../sql/streaming/ui/StreamingQueryTab.scala | 3 +- .../spark/sql/streaming/ui/UIUtils.scala | 12 +- .../spark-events/local-1596020211915 | 160 ----------------- .../apache/spark/deploy/history/Utils.scala | 40 ----- .../ui/StreamingQueryHistorySuite.scala | 63 ------- .../ui/StreamingQueryPageSuite.scala | 42 ++--- .../StreamingQueryStatusListenerSuite.scala | 159 +++-------------- 17 files changed, 158 insertions(+), 693 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala delete mode 100644 sql/core/src/test/resources/spark-events/local-1596020211915 delete mode 100644 sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 167cf224f92c..7da330dfe1fb 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,7 +123,6 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties -local-1596020211915 app-20200706201101-0003 py.typed _metadata diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin index 6771eef52530..0bba2f88b92a 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -1,2 +1 @@ org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin -org.apache.spark.sql.execution.ui.StreamingQueryHistoryServerPlugin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 4b98acd16f6f..1b8d69ffb752 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -31,21 +31,16 @@ import org.apache.spark.util.ListenerBus * Spark listener bus, so that it can receive [[StreamingQueryListener.Event]]s and dispatch them * to StreamingQueryListeners. * - * Note 1: Each bus and its registered listeners are associated with a single SparkSession + * Note that each bus and its registered listeners are associated with a single SparkSession * and StreamingQueryManager. So this bus will dispatch events to registered listeners for only * those queries that were started in the associated SparkSession. - * - * Note 2: To rebuild Structured Streaming UI in SHS, this bus will be registered into - * [[org.apache.spark.scheduler.ReplayListenerBus]]. We check `sparkListenerBus` defined or not to - * determine how to process [[StreamingQueryListener.Event]]. If false, it means this bus is used to - * replay all streaming query event from eventLog. */ -class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) +class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) extends SparkListener with ListenerBus[StreamingQueryListener, StreamingQueryListener.Event] { import StreamingQueryListener._ - sparkListenerBus.foreach(_.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY)) + sparkListenerBus.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY) /** * RunIds of active queries whose events are supposed to be forwarded by this ListenerBus @@ -72,11 +67,11 @@ class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) event match { case s: QueryStartedEvent => activeQueryRunIds.synchronized { activeQueryRunIds += s.runId } - sparkListenerBus.foreach(bus => bus.post(s)) + sparkListenerBus.post(s) // post to local listeners to trigger callbacks postToAll(s) case _ => - sparkListenerBus.foreach(bus => bus.post(event)) + sparkListenerBus.post(event) } } @@ -100,11 +95,7 @@ class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) // synchronously and the ones attached to LiveListenerBus asynchronously. Therefore, // we need to ignore QueryStartedEvent if this method is called within SparkListenerBus // thread - // - // When loaded by Spark History Server, we should process all event coming from replay - // listener bus. - if (sparkListenerBus.isEmpty || !LiveListenerBus.withinListenerThread.value || - !e.isInstanceOf[QueryStartedEvent]) { + if (!LiveListenerBus.withinListenerThread.value || !e.isInstanceOf[QueryStartedEvent]) { postToAll(e) } case _ => @@ -119,10 +110,7 @@ class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) listener: StreamingQueryListener, event: StreamingQueryListener.Event): Unit = { def shouldReport(runId: UUID): Boolean = { - // When loaded by Spark History Server, we should process all event coming from replay - // listener bus. - sparkListenerBus.isEmpty || - activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } + activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } } event match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala deleted file mode 100644 index a127fa59b743..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.ui - -import org.apache.spark.SparkConf -import org.apache.spark.scheduler.SparkListener -import org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus -import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} -import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} -import org.apache.spark.ui.SparkUI - -class StreamingQueryHistoryServerPlugin extends AppHistoryServerPlugin { - - override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { - val listenerBus = new StreamingQueryListenerBus(None) - listenerBus.addListener(new StreamingQueryStatusListener(conf, store)) - Seq(listenerBus) - } - - override def setupUI(ui: SparkUI): Unit = { - val streamingQueryStatusStore = new StreamingQueryStatusStore(ui.store.store) - if (streamingQueryStatusStore.allQueryUIData.nonEmpty) { - new StreamingQueryTab(streamingQueryStatusStore, ui) - } - } - - override def displayOrder: Int = 1 -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala deleted file mode 100644 index 9eb14a6a6306..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.ui - -import java.util.UUID - -import org.apache.spark.sql.streaming.ui.{StreamingQueryData, StreamingQueryProgressWrapper, StreamingQueryUIData} -import org.apache.spark.status.KVUtils -import org.apache.spark.util.kvstore.KVStore - -/** - * Provides a view of a KVStore with methods that make it easy to query Streaming Query state. - * There's no state kept in this class, so it's ok to have multiple instances of it in an - * application. - */ -class StreamingQueryStatusStore(store: KVStore) { - - def allQueryUIData: Seq[StreamingQueryUIData] = { - val view = store.view(classOf[StreamingQueryData]).index("startTimestamp").first(0L) - KVUtils.viewToSeq(view, Int.MaxValue)(_ => true).map(makeUIData) - } - - // visible for test - private[sql] def getQueryProgressData(runId: UUID): Seq[StreamingQueryProgressWrapper] = { - val view = store.view(classOf[StreamingQueryProgressWrapper]) - .index("runId").first(runId.toString).last(runId.toString) - KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) - } - - private def makeUIData(summary: StreamingQueryData): StreamingQueryUIData = { - val runId = summary.runId.toString - val view = store.view(classOf[StreamingQueryProgressWrapper]) - .index("runId").first(runId).last(runId) - val recentProgress = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) - .map(_.progress).sortBy(_.timestamp).toArray - StreamingQueryUIData(summary, recentProgress) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index ea430db9f030..89aceacac600 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab, StreamingQueryStatusStore} +import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} import org.apache.spark.status.ElementTrackingStore @@ -111,9 +111,9 @@ private[sql] class SharedState( lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] = { sparkContext.ui.flatMap { ui => if (conf.get(STREAMING_UI_ENABLED)) { - val kvStore = sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] - new StreamingQueryTab(new StreamingQueryStatusStore(kvStore), ui) - Some(new StreamingQueryStatusListener(conf, kvStore)) + val statusListener = new StreamingQueryStatusListener(conf) + new StreamingQueryTab(statusListener, ui) + Some(statusListener) } else { None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index b66037d00919..ffdbe9d4e491 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -49,8 +49,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) - private val listenerBus = - new StreamingQueryListenerBus(Some(sparkSession.sparkContext.listenerBus)) + private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) @GuardedBy("activeQueriesSharedLock") private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala index 96e498991e1b..b98fdf16eef3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -40,8 +40,8 @@ private[ui] class StreamingQueryPage(parent: StreamingQueryTab) } private def generateStreamingQueryTable(request: HttpServletRequest): Seq[Node] = { - val (activeQueries, inactiveQueries) = - parent.store.allQueryUIData.partition(_.summary.isActive) + val (activeQueries, inactiveQueries) = parent.statusListener.allQueryStatus + .partition(_.isActive) val content = mutable.ListBuffer[Node]() // show active queries table only if there is at least one active query @@ -176,7 +176,7 @@ class StreamingQueryPagedTable( val streamingQuery = query.streamingUIData val statisticsLink = "%s/%s/statistics?id=%s" .format(SparkUIUtils.prependBaseUri(request, parent.basePath), parent.prefix, - streamingQuery.summary.runId) + streamingQuery.runId) def details(detail: Any): Seq[Node] = { if (isActive) { @@ -194,14 +194,14 @@ class StreamingQueryPagedTable( {UIUtils.getQueryName(streamingQuery)} {UIUtils.getQueryStatus(streamingQuery)} - {streamingQuery.summary.id} - {streamingQuery.summary.runId} - {SparkUIUtils.formatDate(streamingQuery.summary.startTimestamp)} + {streamingQuery.id} + {streamingQuery.runId} + {SparkUIUtils.formatDate(streamingQuery.startTimestamp)} {SparkUIUtils.formatDurationVerbose(query.duration)} {withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} - {details(streamingQuery.summary.exception.getOrElse("-"))} + {details(streamingQuery.exception.getOrElse("-"))} } } @@ -222,32 +222,32 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St override def sliceData(from: Int, to: Int): Seq[StructuredStreamingRow] = data.slice(from, to) - private def streamingRow(uiData: StreamingQueryUIData): StructuredStreamingRow = { + private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = { val duration = if (isActive) { - System.currentTimeMillis() - uiData.summary.startTimestamp + System.currentTimeMillis() - query.startTimestamp } else { - withNoProgress(uiData, { - val endTimeMs = uiData.lastProgress.timestamp - parseProgressTimestamp(endTimeMs) - uiData.summary.startTimestamp + withNoProgress(query, { + val endTimeMs = query.lastProgress.timestamp + parseProgressTimestamp(endTimeMs) - query.startTimestamp }, 0) } - val avgInput = (uiData.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / - uiData.recentProgress.length) + val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / + query.recentProgress.length) - val avgProcess = (uiData.recentProgress.map(p => - withNumberInvalid(p.processedRowsPerSecond)).sum / uiData.recentProgress.length) + val avgProcess = (query.recentProgress.map(p => + withNumberInvalid(p.processedRowsPerSecond)).sum / query.recentProgress.length) - StructuredStreamingRow(duration, avgInput, avgProcess, uiData) + StructuredStreamingRow(duration, avgInput, avgProcess, query) } private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = { val ordering: Ordering[StructuredStreamingRow] = sortColumn match { - case "Name" => Ordering.by(row => UIUtils.getQueryName(row.streamingUIData)) - case "Status" => Ordering.by(row => UIUtils.getQueryStatus(row.streamingUIData)) - case "ID" => Ordering.by(_.streamingUIData.summary.id) - case "Run ID" => Ordering.by(_.streamingUIData.summary.runId) - case "Start Time" => Ordering.by(_.streamingUIData.summary.startTimestamp) + case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData)) + case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData)) + case "ID" => Ordering.by(_.streamingUIData.id) + case "Run ID" => Ordering.by(_.streamingUIData.runId) + case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp) case "Duration" => Ordering.by(_.duration) case "Avg Input /sec" => Ordering.by(_.avgInput) case "Avg Process /sec" => Ordering.by(_.avgProcess) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala index 97691d9d7e82..24709ba470cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala @@ -58,8 +58,8 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val query = parent.store.allQueryUIData.find { uiData => - uiData.summary.runId.equals(UUID.fromString(parameterId)) + val query = parent.statusListener.allQueryStatus.find { case q => + q.runId.equals(UUID.fromString(parameterId)) }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming query $parameterId")) val resources = generateLoadResources(request) @@ -109,35 +109,34 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) } - def generateBasicInfo(uiData: StreamingQueryUIData): Seq[Node] = { - val duration = if (uiData.summary.isActive) { - val durationMs = System.currentTimeMillis() - uiData.summary.startTimestamp - SparkUIUtils.formatDurationVerbose(durationMs) + def generateBasicInfo(query: StreamingQueryUIData): Seq[Node] = { + val duration = if (query.isActive) { + SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp) } else { - withNoProgress(uiData, { - val end = uiData.lastProgress.timestamp - val start = uiData.recentProgress.head.timestamp + withNoProgress(query, { + val end = query.lastProgress.timestamp + val start = query.recentProgress.head.timestamp SparkUIUtils.formatDurationVerbose( parseProgressTimestamp(end) - parseProgressTimestamp(start)) }, "-") } - val name = UIUtils.getQueryName(uiData) - val numBatches = withNoProgress(uiData, { uiData.lastProgress.batchId + 1L }, 0) + val name = UIUtils.getQueryName(query) + val numBatches = withNoProgress(query, { query.lastProgress.batchId + 1L }, 0)
Running batches for {duration} since - {SparkUIUtils.formatDate(uiData.summary.startTimestamp)} + {SparkUIUtils.formatDate(query.startTimestamp)} ({numBatches} completed batches)

Name: {name}
-
Id: {uiData.summary.id}
-
RunId: {uiData.summary.runId}
+
Id: {query.id}
+
RunId: {query.runId}

} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala index fdd375434410..e331083b3002 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala @@ -20,144 +20,102 @@ package org.apache.spark.sql.streaming.ui import java.util.UUID import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters._ import scala.collection.mutable -import com.fasterxml.jackson.annotation.JsonIgnore - import org.apache.spark.SparkConf import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress} -import org.apache.spark.sql.streaming.ui.StreamingQueryProgressWrapper._ import org.apache.spark.sql.streaming.ui.UIUtils.parseProgressTimestamp -import org.apache.spark.status.{ElementTrackingStore, KVUtils} -import org.apache.spark.status.KVUtils.KVIndexParam -import org.apache.spark.util.kvstore.KVIndex /** * A customized StreamingQueryListener used in structured streaming UI, which contains all * UI data for both active and inactive query. + * TODO: Add support for history server. */ -private[sql] class StreamingQueryStatusListener( - conf: SparkConf, - store: ElementTrackingStore) extends StreamingQueryListener { +private[sql] class StreamingQueryStatusListener(conf: SparkConf) extends StreamingQueryListener { + + /** + * We use runId as the key here instead of id in active query status map, + * because the runId is unique for every started query, even it its a restart. + */ + private[ui] val activeQueryStatus = new ConcurrentHashMap[UUID, StreamingQueryUIData]() + private[ui] val inactiveQueryStatus = new mutable.Queue[StreamingQueryUIData]() private val streamingProgressRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES) private val inactiveQueryStatusRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES) - store.addTrigger(classOf[StreamingQueryData], inactiveQueryStatusRetention) { count => - cleanupInactiveQueries(count) - } - - // Events from the same query run will never be processed concurrently, so it's safe to - // access `progressIds` without any protection. - private val queryToProgress = new ConcurrentHashMap[UUID, mutable.Queue[String]]() - - private def cleanupInactiveQueries(count: Long): Unit = { - val view = store.view(classOf[StreamingQueryData]).index("active").first(false).last(false) - val inactiveQueries = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) - val numInactiveQueries = inactiveQueries.size - if (numInactiveQueries <= inactiveQueryStatusRetention) { - return - } - val toDelete = inactiveQueries.sortBy(_.endTimestamp.get) - .take(numInactiveQueries - inactiveQueryStatusRetention) - val runIds = toDelete.map { e => - store.delete(e.getClass, e.runId) - e.runId.toString - } - // Delete wrappers in one pass, as deleting them for each summary is slow - store.removeAllByIndexValues(classOf[StreamingQueryProgressWrapper], "runId", runIds) - } - override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { val startTimestamp = parseProgressTimestamp(event.timestamp) - store.write(new StreamingQueryData( - event.name, - event.id, - event.runId, - isActive = true, - None, - startTimestamp - ), checkTriggers = true) + activeQueryStatus.putIfAbsent(event.runId, + new StreamingQueryUIData(event.name, event.id, event.runId, startTimestamp)) } override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { - val runId = event.progress.runId - val batchId = event.progress.batchId - val timestamp = event.progress.timestamp - if (!queryToProgress.containsKey(runId)) { - queryToProgress.put(runId, mutable.Queue.empty[String]) - } - val progressIds = queryToProgress.get(runId) - progressIds.enqueue(getUniqueId(runId, batchId, timestamp)) - store.write(new StreamingQueryProgressWrapper(event.progress)) - while (progressIds.length > streamingProgressRetention) { - val uniqueId = progressIds.dequeue - store.delete(classOf[StreamingQueryProgressWrapper], uniqueId) - } + val batchTimestamp = parseProgressTimestamp(event.progress.timestamp) + val queryStatus = activeQueryStatus.getOrDefault( + event.progress.runId, + new StreamingQueryUIData(event.progress.name, event.progress.id, event.progress.runId, + batchTimestamp)) + queryStatus.updateProcess(event.progress, streamingProgressRetention) } override def onQueryTerminated( - event: StreamingQueryListener.QueryTerminatedEvent): Unit = { - val querySummary = store.read(classOf[StreamingQueryData], event.runId) - val curTime = System.currentTimeMillis() - store.write(new StreamingQueryData( - querySummary.name, - querySummary.id, - querySummary.runId, - isActive = false, - querySummary.exception, - querySummary.startTimestamp, - Some(curTime) - ), checkTriggers = true) - queryToProgress.remove(event.runId) + event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { + val queryStatus = activeQueryStatus.remove(event.runId) + if (queryStatus != null) { + queryStatus.queryTerminated(event) + inactiveQueryStatus += queryStatus + while (inactiveQueryStatus.length >= inactiveQueryStatusRetention) { + inactiveQueryStatus.dequeue() + } + } } -} -private[sql] class StreamingQueryData( - val name: String, - val id: UUID, - @KVIndexParam val runId: UUID, - @KVIndexParam("active") val isActive: Boolean, - val exception: Option[String], - @KVIndexParam("startTimestamp") val startTimestamp: Long, - val endTimestamp: Option[Long] = None) + def allQueryStatus: Seq[StreamingQueryUIData] = synchronized { + activeQueryStatus.values().asScala.toSeq ++ inactiveQueryStatus + } +} /** * This class contains all message related to UI display, each instance corresponds to a single * [[org.apache.spark.sql.streaming.StreamingQuery]]. */ -private[sql] case class StreamingQueryUIData( - summary: StreamingQueryData, - recentProgress: Array[StreamingQueryProgress]) { - - def lastProgress: StreamingQueryProgress = { - if (recentProgress.nonEmpty) { - recentProgress.last - } else { - null - } +private[ui] class StreamingQueryUIData( + val name: String, + val id: UUID, + val runId: UUID, + val startTimestamp: Long) { + + /** Holds the most recent query progress updates. */ + private val progressBuffer = new mutable.Queue[StreamingQueryProgress]() + + private var _isActive = true + private var _exception: Option[String] = None + + def isActive: Boolean = synchronized { _isActive } + + def exception: Option[String] = synchronized { _exception } + + def queryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { + _isActive = false + _exception = event.exception } -} -private[sql] class StreamingQueryProgressWrapper(val progress: StreamingQueryProgress) { - @JsonIgnore @KVIndex - private val uniqueId: String = getUniqueId(progress.runId, progress.batchId, progress.timestamp) + def updateProcess( + newProgress: StreamingQueryProgress, retentionNum: Int): Unit = progressBuffer.synchronized { + progressBuffer += newProgress + while (progressBuffer.length >= retentionNum) { + progressBuffer.dequeue() + } + } - @JsonIgnore @KVIndex("runId") - private def runIdIndex: String = progress.runId.toString -} + def recentProgress: Array[StreamingQueryProgress] = progressBuffer.synchronized { + progressBuffer.toArray + } -private[sql] object StreamingQueryProgressWrapper { - /** - * Adding `timestamp` into unique id to support reporting `empty` query progress - * in which no data comes but with the same batchId. - */ - def getUniqueId( - runId: UUID, - batchId: Long, - timestamp: String): String = { - s"${runId}_${batchId}_$timestamp" + def lastProgress: StreamingQueryProgress = progressBuffer.synchronized { + progressBuffer.lastOption.orNull } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala index 65cad8f06cc1..bb097ffc0691 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql.streaming.ui import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.ui.{SparkUI, SparkUITab} private[sql] class StreamingQueryTab( - val store: StreamingQueryStatusStore, + val statusListener: StreamingQueryStatusListener, sparkUI: SparkUI) extends SparkUITab(sparkUI, "StreamingQuery") with Logging { override val name = "Structured Streaming" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala index 88a110fa9a32..1f7e65dede17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala @@ -46,19 +46,19 @@ private[ui] object UIUtils { } } - def getQueryName(uiData: StreamingQueryUIData): String = { - if (uiData.summary.name == null || uiData.summary.name.isEmpty) { + def getQueryName(query: StreamingQueryUIData): String = { + if (query.name == null || query.name.isEmpty) { "" } else { - uiData.summary.name + query.name } } - def getQueryStatus(uiData: StreamingQueryUIData): String = { - if (uiData.summary.isActive) { + def getQueryStatus(query: StreamingQueryUIData): String = { + if (query.isActive) { "RUNNING" } else { - uiData.summary.exception.map(_ => "FAILED").getOrElse("FINISHED") + query.exception.map(_ => "FAILED").getOrElse("FINISHED") } } diff --git a/sql/core/src/test/resources/spark-events/local-1596020211915 b/sql/core/src/test/resources/spark-events/local-1596020211915 deleted file mode 100644 index ff34bbc16ef3..000000000000 --- a/sql/core/src/test/resources/spark-events/local-1596020211915 +++ /dev/null @@ -1,160 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} -{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1596020212090,"Executor ID":"driver","Executor Info":{"Host":"iZbp19vpr16ix621sdw476Z","Total Cores":4,"Log Urls":{},"Attributes":{},"Resources":{},"Resource Profile Id":0}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Port":39845},"Maximum Memory":384093388,"Timestamp":1596020212109,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","Java Version":"1.8.0_252 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.driver.port":"46309","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","spark.app.name":"StructuredKafkaWordCount","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","fs.s3.block.size":"67108864","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","mapred.child.java.opts":"-Xmx200m","hadoop.common.configuration.version":"0.23.0","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","mapreduce.task.profile.reduces":"0-2","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.nodemanager.webapp.cross-origin.enabled":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.s3a.fast.upload.active.blocks":"4","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","ftp.blocksize":"67108864","yarn.nodemanager.log-container-debug-info.enabled":"false","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.amrmproxy.enable":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"false","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","io.map.index.interval":"128","mapreduce.job.counters.max":"120","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","s3native.bytes-per-checksum":"512","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","yarn.node-labels.fs-store.retry-policy-spec":"2000, 500","hadoop.security.groups.cache.background.reload":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","hadoop.http.authentication.type":"simple","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","yarn.nodemanager.resource-monitor.interval-ms":"3000","mapreduce.shuffle.listen.queue.size":"128","mapreduce.map.cpu.vcores":"1","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.app.mapreduce.client.job.max-retries":"0","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.timeline-service.client.internal-timers-ttl-secs":"420","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"1024","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","fs.s3n.multipart.uploads.block.size":"67108864","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"1024","s3native.client-write-packet-size":"65536","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","yarn.sharedcache.nm.uploader.thread-count":"20","mapreduce.shuffle.ssl.enabled":"false","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"json","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","yarn.resourcemanager.zk-num-retries":"1000","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8048","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME","yarn.sharedcache.nested-level":"3","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false"},"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":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/amd64","user.dir":"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.252-b09","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_252-b09","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"Asia/Shanghai","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-1127.10.1.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","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":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[*] --conf spark.eventLog.dir=/tmp/spark-history --conf spark.eventLog.enabled=true --conf spark.sql.shuffle.partitions=2 --class org.apache.spark.examples.sql.streaming.StructuredKafkaWordCount ./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar 192.168.130.97:9092 subscribe test5","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","java.version":"1.8.0_252","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-graphite-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-server-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/pyrolite-4.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/conf/":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-smart-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/objenesis-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-auth-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsp-api-2.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-codec-1.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/protobuf-java-2.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/transaction-api-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/gson-2.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libfb303-0.9.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-cli-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-library-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-api-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okhttp-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/derby-10.12.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-collections-3.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpcore-4.4.12.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-beanutils-1.9.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-crypto-1.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-ast_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/lz4-java-1.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-format-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-column-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-logging-1.1.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/audience-annotations-0.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-jdbc-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive-thriftserver_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-cli-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javolution-5.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JLargeArrays-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-api-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-dbcp-1.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/threeten-extra-1.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-io-2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-json-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libthrift-0.12.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/compress-lzf-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jmx-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.inject-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-recipes-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-api-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/oro-2.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-memory-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jpam-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/velocity-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-core-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-databind-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-text-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-client-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-ipc-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-util-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/core-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/joda-time-2.10.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-encoding-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-llap-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/paranamer-2.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/activation-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-framework-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compress-1.8.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/RoaringBitmap-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ivy-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-httpclient-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-yarn_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-common-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zstd-jni-1.4.5-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/snappy-java-1.1.7.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool-1.5.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-core_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpclient-4.5.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-util-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aircompressor-0.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/leveldbjni-all-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-hk2-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jta-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-net-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-core-4.1.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-web-proxy-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-mapreduce-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xz-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.inject-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-compiler-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jvm-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-shims-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-api-2.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/janino-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-app-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-utils-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JTransforms-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kafka-clients-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-servlet-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okio-1.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-annotations-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-math3-3.4.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-scalap_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/accessors-smart-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guava-14.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shims-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ST4-4.0.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill-java-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-shuffle-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stream-2.9.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-configuration-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jodd-core-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool2-2.6.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xmlenc-0.52.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-token-provider-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-jackson_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compiler-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jline-2.14.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kryo-shaded-4.0.2.jar":"System Classpath","spark://iZbp19vpr16ix621sdw476Z:46309/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar":"Added By User","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-common-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-client-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-xc-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zookeeper-3.4.14.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/univocity-parsers-2.8.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-digester-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arpack_combined_all-0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-annotations-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-locator-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-core-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-reflect-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/super-csv-2.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-jobclient-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-exec-2.3.7-core.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/opencsv-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-storage-api-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/HikariCP-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-metastore-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/minlog-1.3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-format-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsr305-3.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang-2.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang3-3.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javassist-3.25.0-GA.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr-runtime-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/log4j-1.2.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-beeline-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill_2.12-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jdo-api-3.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-core-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/py4j-0.10.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-serde-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-hdfs-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcip-annotations-1.0-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-jackson-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/netty-all-4.1.47.Final.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-vector-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/machinist_2.12-0.6.8.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"StructuredKafkaWordCount","App ID":"local-1596020211915","Timestamp":1596020210919,"User":"root"} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryStartedEvent","id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:55.947Z"} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220179} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#218]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220258} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1596020221633,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0,1],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222649,"Failed":false,"Killed":false,"Accumulables":[{"ID":21,"Name":"shuffle write time","Update":"9599308","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Update":"296","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Update":"200","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"time in aggregation build","Update":"190","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"duration","Update":"336","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Update":9599308,"Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":17,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Update":2630,"Value":2630,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorCpuTime","Update":466139164,"Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Update":503,"Value":503,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Update":301869581,"Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Update":361,"Value":361,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":361,"Executor Deserialize CPU Time":301869581,"Executor Run Time":503,"Executor CPU Time":466139164,"Peak Execution Memory":524288,"Result Size":2630,"JVM GC Time":17,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":9599308,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Completion Time":1596020222661,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"time in aggregation build","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Value":2630,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":106,"Name":"internal.metrics.executorCpuTime","Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Value":361,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"duration","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Value":503,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":108,"Name":"internal.metrics.jvmGCTime","Value":17,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"shuffle write time","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222954,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"50","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"64","Value":"64","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"208","Value":"208","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5354,"Value":5354,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":93367533,"Value":93367533,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":203,"Value":203,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10308753,"Value":10308753,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":10308753,"Executor Run Time":203,"Executor CPU Time":93367533,"Peak Execution Memory":524288,"Result Size":5354,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222965,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"33","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":45,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":53,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"31","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"424","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":50,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"568","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5574,"Value":10928,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":91355172,"Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":205,"Value":408,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":21029530,"Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":34,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":34,"Executor Deserialize CPU Time":21029530,"Executor Run Time":205,"Executor CPU Time":91355172,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Completion Time":1596020222967,"Accumulables":[{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":50,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"duration","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"estimated size of state only on current version","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":130,"Name":"internal.metrics.executorRunTime","Value":408,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Value":10928,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1596020222973,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#46, count#47]\nArguments: [value#46, count#47]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#46, count#47]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":153,"metricType":"sum"}]},"time":1596020223028} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1596020223062} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1596020223069} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1596020223069} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:56.015Z","batchId":0,"batchDuration":7110,"durationMs":{"triggerExecution":7109,"queryPlanning":439,"getBatch":21,"latestOffset":3524,"addBatch":3011,"walCommit":35},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":776,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":488,"loadedMapCacheHitCount":0,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":null,"endOffset":"{\"test5\":{\"0\":48279}}","numInputRows":3,"inputRowsPerSecond":"NaN","processedRowsPerSecond":0.42194092827004215}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223333} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#449]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223382} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1596020223482,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2,3],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223601,"Failed":false,"Killed":false,"Accumulables":[{"ID":178,"Name":"shuffle write time","Update":"837580","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":226,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"time in aggregation build","Update":"74","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":234,"Name":"time in aggregation build","Update":"68","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":237,"Name":"number of output rows","Update":"363","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":282,"Name":"internal.metrics.input.recordsRead","Update":363,"Value":363,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Update":837580,"Value":837580,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.executorCpuTime","Update":95945587,"Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.executorRunTime","Update":96,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7437557,"Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7437557,"Executor Run Time":96,"Executor CPU Time":95945587,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":169,"Shuffle Write Time":837580,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":363},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Completion Time":1596020223603,"Accumulables":[{"ID":227,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":262,"Name":"internal.metrics.executorRunTime","Value":96,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Value":837580,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"time in aggregation build","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"shuffle write time","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"time in aggregation build","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":264,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":282,"Name":"internal.metrics.input.recordsRead","Value":363,"Internal":true,"Count Failed Values":true},{"ID":237,"Name":"number of output rows","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":263,"Name":"internal.metrics.executorCpuTime","Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":169,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223717,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"376","Value":"376","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":22954307,"Value":22954307,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":77,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6627382,"Value":6627382,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6627382,"Executor Run Time":77,"Executor CPU Time":22954307,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223720,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":206,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"18","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"30","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"840","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"19","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":175,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":25907369,"Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":82,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7573630,"Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":13,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7573630,"Executor Run Time":82,"Executor CPU Time":25907369,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":169,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Completion Time":1596020223724,"Accumulables":[{"ID":218,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":175,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":169,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"count of cache hit on states cache in provider","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":201,"Name":"duration","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":287,"Name":"internal.metrics.executorRunTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1596020223725,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#60, count#61]\nArguments: [value#60, count#61]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#60, count#61]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":310,"metricType":"sum"}]},"time":1596020223752} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1596020223761} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1596020223762} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1596020223762} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.168Z","batchId":1,"batchDuration":622,"durationMs":{"triggerExecution":622,"queryPlanning":47,"getBatch":0,"latestOffset":7,"addBatch":478,"walCommit":59},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1216,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":4,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48279}}","endOffset":"{\"test5\":{\"0\":48642}}","numInputRows":363,"inputRowsPerSecond":50.74793792814204,"processedRowsPerSecond":583.6012861736334}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223909} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#680]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224006} -{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1596020224100,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[5,4],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224174,"Failed":false,"Killed":false,"Accumulables":[{"ID":335,"Name":"shuffle write time","Update":"686296","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Update":"39","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":384,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":386,"Name":"time in aggregation build","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":388,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":391,"Name":"time in aggregation build","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Update":"40","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":394,"Name":"number of output rows","Update":"63","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Update":686296,"Value":686296,"Internal":true,"Count Failed Values":true},{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":420,"Name":"internal.metrics.executorCpuTime","Update":33390843,"Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4867521,"Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":4867521,"Executor Run Time":49,"Executor CPU Time":33390843,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":686296,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":63},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Completion Time":1596020224175,"Accumulables":[{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":391,"Name":"time in aggregation build","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":394,"Name":"number of output rows","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":388,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":420,"Name":"internal.metrics.executorCpuTime","Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":384,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Value":686296,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Value":49,"Internal":true,"Count Failed Values":true},{"ID":386,"Name":"time in aggregation build","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":335,"Name":"shuffle write time","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224256,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":17230622,"Value":17230622,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5948051,"Value":5948051,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5948051,"Executor Run Time":56,"Executor CPU Time":17230622,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224257,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"21","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"18","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":364,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"22","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":332,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":23808555,"Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6247106,"Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6247106,"Executor Run Time":56,"Executor CPU Time":23808555,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Completion Time":1596020224259,"Accumulables":[{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"count of cache hit on states cache in provider","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":358,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":369,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":360,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":372,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":444,"Name":"internal.metrics.executorRunTime","Value":112,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1596020224259,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#74, count#75]\nArguments: [value#74, count#75]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#74, count#75]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":467,"metricType":"sum"}]},"time":1596020224278} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1596020224287} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1596020224287} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1596020224288} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.793Z","batchId":2,"batchDuration":522,"durationMs":{"triggerExecution":522,"queryPlanning":41,"getBatch":1,"latestOffset":3,"addBatch":421,"walCommit":27},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":8,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48642}}","endOffset":"{\"test5\":{\"0\":48705}}","numInputRows":63,"inputRowsPerSecond":100.8,"processedRowsPerSecond":120.6896551724138}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":9,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224419} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":10,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#911]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224452} -{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1596020224533,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[6,7],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224581,"Failed":false,"Killed":false,"Accumulables":[{"ID":492,"Name":"shuffle write time","Update":"643278","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":490,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":540,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":543,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":548,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":550,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":551,"Name":"number of output rows","Update":"52","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":596,"Name":"internal.metrics.input.recordsRead","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Update":643278,"Value":643278,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":578,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":577,"Name":"internal.metrics.executorCpuTime","Update":29099071,"Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":576,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3091128,"Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3091128,"Executor Run Time":29,"Executor CPU Time":29099071,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":643278,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":52},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Completion Time":1596020224582,"Accumulables":[{"ID":550,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":577,"Name":"internal.metrics.executorCpuTime","Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":490,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":576,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":540,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Value":643278,"Internal":true,"Count Failed Values":true},{"ID":543,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":492,"Name":"shuffle write time","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":548,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":578,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":596,"Name":"internal.metrics.input.recordsRead","Value":52,"Internal":true,"Count Failed Values":true},{"ID":551,"Name":"number of output rows","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224670,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"27","Value":"27","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":19967906,"Value":19967906,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4899567,"Value":4899567,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4899567,"Executor Run Time":62,"Executor CPU Time":19967906,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224687,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":516,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"26","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":521,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":523,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":484,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":488,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":22402538,"Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":79,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4671511,"Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4671511,"Executor Run Time":79,"Executor CPU Time":22402538,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Completion Time":1596020224688,"Accumulables":[{"ID":523,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":534,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":521,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":515,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":536,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":526,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Value":141,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1596020224689,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":11,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#88, count#89]\nArguments: [value#88, count#89]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#88, count#89]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":624,"metricType":"sum"}]},"time":1596020224709} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":11,"time":1596020224713} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":10,"time":1596020224714} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":9,"time":1596020224714} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.317Z","batchId":3,"batchDuration":415,"durationMs":{"triggerExecution":415,"queryPlanning":38,"getBatch":1,"latestOffset":3,"addBatch":332,"walCommit":21},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":12,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48705}}","endOffset":"{\"test5\":{\"0\":48757}}","numInputRows":52,"inputRowsPerSecond":99.23664122137404,"processedRowsPerSecond":125.30120481927712}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":12,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1066]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224817} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":13,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224849} -{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1596020224928,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[9,8],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224979,"Failed":false,"Killed":false,"Accumulables":[{"ID":649,"Name":"shuffle write time","Update":"572754","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":648,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":696,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":697,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":698,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":700,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":705,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":707,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":708,"Name":"number of output rows","Update":"42","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Update":42,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Update":572754,"Value":572754,"Internal":true,"Count Failed Values":true},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":735,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":734,"Name":"internal.metrics.executorCpuTime","Update":27800373,"Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":733,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4768103,"Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4768103,"Executor Run Time":28,"Executor CPU Time":27800373,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":572754,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":42},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Completion Time":1596020224979,"Accumulables":[{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":696,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":705,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":708,"Name":"number of output rows","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Value":42,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":698,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":734,"Name":"internal.metrics.executorCpuTime","Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":707,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":733,"Name":"internal.metrics.executorRunTime","Value":28,"Internal":true,"Count Failed Values":true},{"ID":697,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Value":572754,"Internal":true,"Count Failed Values":true},{"ID":700,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":649,"Name":"shuffle write time","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225056,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":678,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":690,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":641,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":644,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":646,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":19548688,"Value":19548688,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5622533,"Value":5622533,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":5,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5622533,"Executor Run Time":52,"Executor CPU Time":19548688,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225058,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"35","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":16813539,"Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":55,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4322992,"Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":9,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4322992,"Executor Run Time":55,"Executor CPU Time":16813539,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Completion Time":1596020225059,"Accumulables":[{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":678,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":759,"Name":"internal.metrics.executorCpuTime","Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"count of cache hit on states cache in provider","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":690,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":672,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.executorRunTime","Value":107,"Internal":true,"Count Failed Values":true},{"ID":644,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1596020225059,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":14,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#102, count#103]\nArguments: [value#102, count#103]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#102, count#103]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":781,"metricType":"sum"}]},"time":1596020225079} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":14,"time":1596020225087} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":13,"time":1596020225087} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":12,"time":1596020225087} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.734Z","batchId":4,"batchDuration":387,"durationMs":{"triggerExecution":387,"queryPlanning":30,"getBatch":1,"latestOffset":3,"addBatch":306,"walCommit":12},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":16,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48757}}","endOffset":"{\"test5\":{\"0\":48799}}","numInputRows":42,"inputRowsPerSecond":100.71942446043165,"processedRowsPerSecond":108.52713178294573}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":15,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1297]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225211} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":16,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225270} -{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1596020225342,"Stage Infos":[{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[10,11],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225400,"Failed":false,"Killed":false,"Accumulables":[{"ID":806,"Name":"shuffle write time","Update":"530930","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":855,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":857,"Name":"time in aggregation build","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":864,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":865,"Name":"number of output rows","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":910,"Name":"internal.metrics.input.recordsRead","Update":38,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Update":530930,"Value":530930,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":891,"Name":"internal.metrics.executorCpuTime","Update":22440089,"Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6808170,"Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6808170,"Executor Run Time":29,"Executor CPU Time":22440089,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":530930,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":38},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Completion Time":1596020225401,"Accumulables":[{"ID":855,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":891,"Name":"internal.metrics.executorCpuTime","Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":864,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Value":530930,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":857,"Name":"time in aggregation build","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":806,"Name":"shuffle write time","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":910,"Name":"internal.metrics.input.recordsRead","Value":38,"Internal":true,"Count Failed Values":true},{"ID":865,"Name":"number of output rows","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225498,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":830,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"37","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":802,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":17945299,"Value":17945299,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":68,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3451032,"Value":3451032,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3451032,"Executor Run Time":68,"Executor CPU Time":17945299,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225509,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"50","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":15599091,"Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":84,"Value":152,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4357806,"Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4357806,"Executor Run Time":84,"Executor CPU Time":15599091,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Completion Time":1596020225514,"Accumulables":[{"ID":846,"Name":"duration","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":830,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":833,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":844,"Name":"count of cache hit on states cache in provider","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":916,"Name":"internal.metrics.executorCpuTime","Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":829,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":840,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":915,"Name":"internal.metrics.executorRunTime","Value":152,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1596020225514,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":17,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#116, count#117]\nArguments: [value#116, count#117]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#116, count#117]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":938,"metricType":"sum"}]},"time":1596020225536} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":17,"time":1596020225541} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":16,"time":1596020225542} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":15,"time":1596020225542} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.123Z","batchId":5,"batchDuration":437,"durationMs":{"triggerExecution":437,"queryPlanning":35,"getBatch":1,"latestOffset":3,"addBatch":361,"walCommit":18},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":20,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48799}}","endOffset":"{\"test5\":{\"0\":48837}}","numInputRows":38,"inputRowsPerSecond":97.68637532133675,"processedRowsPerSecond":86.95652173913044}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":18,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1528]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225657} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":19,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225687} -{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1596020225759,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[12,13],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225796,"Failed":false,"Killed":false,"Accumulables":[{"ID":963,"Name":"shuffle write time","Update":"543836","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1010,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1012,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1014,"Name":"time in aggregation build","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1016,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1019,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1022,"Name":"number of output rows","Update":"44","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Update":44,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Update":543836,"Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1049,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Update":23733439,"Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3714406,"Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3714406,"Executor Run Time":23,"Executor CPU Time":23733439,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543836,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":44},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Completion Time":1596020225797,"Accumulables":[{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1010,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1019,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Value":44,"Internal":true,"Count Failed Values":true},{"ID":1022,"Name":"number of output rows","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1049,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1016,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1012,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Value":23,"Internal":true,"Count Failed Values":true},{"ID":1014,"Name":"time in aggregation build","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":963,"Name":"shuffle write time","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225868,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17503528,"Value":17503528,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4255703,"Value":4255703,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4255703,"Executor Run Time":50,"Executor CPU Time":17503528,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225874,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":991,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":987,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"23","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":992,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":955,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":960,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17516707,"Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":56,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3750230,"Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3750230,"Executor Run Time":56,"Executor CPU Time":17516707,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Completion Time":1596020225874,"Accumulables":[{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":992,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1001,"Name":"count of cache hit on states cache in provider","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":986,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1007,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":997,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":955,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":991,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1072,"Name":"internal.metrics.executorRunTime","Value":106,"Internal":true,"Count Failed Values":true},{"ID":1000,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1074,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":960,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":987,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":999,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1596020225875,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":20,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#130, count#131]\nArguments: [value#130, count#131]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#130, count#131]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1095,"metricType":"sum"}]},"time":1596020225891} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":20,"time":1596020225896} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":19,"time":1596020225897} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":18,"time":1596020225897} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.562Z","batchId":6,"batchDuration":351,"durationMs":{"triggerExecution":351,"queryPlanning":28,"getBatch":1,"latestOffset":6,"addBatch":273,"walCommit":25},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":24,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48837}}","endOffset":"{\"test5\":{\"0\":48881}}","numInputRows":44,"inputRowsPerSecond":100.22779043280183,"processedRowsPerSecond":125.35612535612536}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":21,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1759]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225988} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":22,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020226019} -{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1596020226076,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[15,14],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226116,"Failed":false,"Killed":false,"Accumulables":[{"ID":1120,"Name":"shuffle write time","Update":"543034","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1167,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1171,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1176,"Name":"time in aggregation build","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1178,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1179,"Name":"number of output rows","Update":"36","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Update":543034,"Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1206,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1205,"Name":"internal.metrics.executorCpuTime","Update":19652237,"Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1204,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2829254,"Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2829254,"Executor Run Time":19,"Executor CPU Time":19652237,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543034,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":36},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Completion Time":1596020226117,"Accumulables":[{"ID":1205,"Name":"internal.metrics.executorCpuTime","Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1178,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1204,"Name":"internal.metrics.executorRunTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1171,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1120,"Name":"shuffle write time","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1167,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1176,"Name":"time in aggregation build","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1206,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Value":36,"Internal":true,"Count Failed Values":true},{"ID":1179,"Name":"number of output rows","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226196,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1148,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1152,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1112,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1116,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1115,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1117,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":19415818,"Value":19415818,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":60,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3845429,"Value":3845429,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3845429,"Executor Run Time":60,"Executor CPU Time":19415818,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226204,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"48","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":14652861,"Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":65,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3933877,"Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3933877,"Executor Run Time":65,"Executor CPU Time":14652861,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Completion Time":1596020226204,"Accumulables":[{"ID":1115,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1231,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1153,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1117,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1147,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":1158,"Name":"count of cache hit on states cache in provider","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1152,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1116,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1143,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1148,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Value":125,"Internal":true,"Count Failed Values":true},{"ID":1112,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} -{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1596020226204,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":23,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#144, count#145]\nArguments: [value#144, count#145]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#144, count#145]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1252,"metricType":"sum"}]},"time":1596020226221} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":23,"time":1596020226230} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":22,"time":1596020226231} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":21,"time":1596020226231} -{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.916Z","batchId":7,"batchDuration":341,"durationMs":{"triggerExecution":341,"queryPlanning":24,"getBatch":0,"latestOffset":3,"addBatch":271,"walCommit":14},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":28,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48881}}","endOffset":"{\"test5\":{\"0\":48917}}","numInputRows":36,"inputRowsPerSecond":101.69491525423729,"processedRowsPerSecond":105.57184750733137}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1596020226301} diff --git a/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala b/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala deleted file mode 100644 index f73305b1b001..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.history - -import org.apache.spark.SparkConf -import org.apache.spark.internal.config.History.HISTORY_LOG_DIR -import org.apache.spark.util.ManualClock - -object Utils { - def withFsHistoryProvider(logDir: String)(fn: FsHistoryProvider => Unit): Unit = { - var provider: FsHistoryProvider = null - try { - val clock = new ManualClock() - val conf = new SparkConf().set(HISTORY_LOG_DIR, logDir) - val provider = new FsHistoryProvider(conf, clock) - provider.checkForLogs() - fn(provider) - } finally { - if (provider != null) { - provider.stop() - provider = null - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala deleted file mode 100644 index 160535ea4d04..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming.ui - -import java.util.Locale -import javax.servlet.http.HttpServletRequest - -import org.mockito.Mockito.{mock, when} -import org.scalatest.BeforeAndAfter - -import org.apache.spark.deploy.history.{Utils => HsUtils} -import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore -import org.apache.spark.sql.test.SharedSparkSession - -class StreamingQueryHistorySuite extends SharedSparkSession with BeforeAndAfter { - - test("support streaming query events") { - val logDir = Thread.currentThread().getContextClassLoader.getResource("spark-events").toString - HsUtils.withFsHistoryProvider(logDir) { provider => - val appUi = provider.getAppUI("local-1596020211915", None).getOrElse { - assert(false, "Failed to load event log of local-1596020211915.") - null - } - assert(appUi.ui.appName == "StructuredKafkaWordCount") - assert(appUi.ui.store.store.count(classOf[StreamingQueryData]) == 1) - assert(appUi.ui.store.store.count(classOf[StreamingQueryProgressWrapper]) == 8) - - val store = new StreamingQueryStatusStore(appUi.ui.store.store) - val tab = new StreamingQueryTab(store, appUi.ui) - val request = mock(classOf[HttpServletRequest]) - var html = new StreamingQueryPage(tab).render(request) - .toString().toLowerCase(Locale.ROOT) - // 81.39: Avg Input /sec - assert(html.contains("81.39")) - // 157.05: Avg Process /sec - assert(html.contains("157.05")) - - val id = "8d268dc2-bc9c-4be8-97a9-b135d2943028" - val runId = "e225d92f-2545-48f8-87a2-9c0309580f8a" - when(request.getParameter("id")).thenReturn(runId) - html = new StreamingQueryStatisticsPage(tab).render(request) - .toString().toLowerCase(Locale.ROOT) - assert(html.contains("8 completed batches")) - assert(html.contains(id)) - assert(html.contains(runId)) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala index 246fa1f7c918..c2b6688faf0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala @@ -20,13 +20,11 @@ package org.apache.spark.sql.streaming.ui import java.util.{Locale, UUID} import javax.servlet.http.HttpServletRequest -import scala.xml.Node - import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter +import scala.xml.Node import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.sql.streaming.StreamingQueryProgress import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.ui.SparkUI @@ -37,26 +35,26 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) + val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.store).thenReturn(store) + when(tab.statusListener).thenReturn(statusListener) val streamQuery = createStreamQueryUIData(id) - when(store.allQueryUIData).thenReturn(Seq(streamQuery)) + when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) var html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("active streaming queries (1)")) - when(streamQuery.summary.isActive).thenReturn(false) - when(streamQuery.summary.exception).thenReturn(None) + when(streamQuery.isActive).thenReturn(false) + when(streamQuery.exception).thenReturn(None) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) assert(html.contains("finished")) - when(streamQuery.summary.isActive).thenReturn(false) - when(streamQuery.summary.exception).thenReturn(Option("exception in query")) + when(streamQuery.isActive).thenReturn(false) + when(streamQuery.exception).thenReturn(Option("exception in query")) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) @@ -68,20 +66,17 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) - when(request.getParameter("id")).thenReturn(id.toString) - when(tab.appName).thenReturn("testing") - when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.store).thenReturn(store) + val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) val ui = mock(classOf[SparkUI]) when(request.getParameter("id")).thenReturn(id.toString) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) + when(tab.statusListener).thenReturn(statusListener) when(ui.conf).thenReturn(new SparkConf()) when(tab.parent).thenReturn(ui) val streamQuery = createStreamQueryUIData(id) - when(store.allQueryUIData).thenReturn(Seq(streamQuery)) + when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) val html = renderStreamingQueryStatisticsPage(request, tab) .toString().toLowerCase(Locale.ROOT) @@ -99,18 +94,15 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { when(progress.batchId).thenReturn(2) when(progress.prettyJson).thenReturn("""{"a":1}""") - val summary = mock(classOf[StreamingQueryData], RETURNS_SMART_NULLS) - when(summary.isActive).thenReturn(true) - when(summary.name).thenReturn("query") - when(summary.id).thenReturn(id) - when(summary.runId).thenReturn(id) - when(summary.startTimestamp).thenReturn(1L) - when(summary.exception).thenReturn(None) - val streamQuery = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) - when(streamQuery.summary).thenReturn(summary) + when(streamQuery.isActive).thenReturn(true) + when(streamQuery.name).thenReturn("query") + when(streamQuery.id).thenReturn(id) + when(streamQuery.runId).thenReturn(id) + when(streamQuery.startTimestamp).thenReturn(1L) when(streamQuery.lastProgress).thenReturn(progress) when(streamQuery.recentProgress).thenReturn(Array(progress)) + when(streamQuery.exception).thenReturn(None) streamQuery } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala index 91c55d5598a6..6aa440e5609c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala @@ -17,28 +17,19 @@ package org.apache.spark.sql.streaming.ui -import java.text.SimpleDateFormat -import java.util.{Date, UUID} +import java.util.UUID import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} -import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone -import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore -import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress, StreamTest} import org.apache.spark.sql.streaming -import org.apache.spark.status.ElementTrackingStore -import org.apache.spark.util.kvstore.InMemoryStore class StreamingQueryStatusListenerSuite extends StreamTest { test("onQueryStarted, onQueryProgress, onQueryTerminated") { - val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) - val queryStore = new StreamingQueryStatusStore(kvStore) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) - // handle query started event + // hanlde query started event val id = UUID.randomUUID() val runId = UUID.randomUUID() val startEvent = new StreamingQueryListener.QueryStartedEvent( @@ -46,9 +37,8 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent) // result checking - assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) - assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => - uiData.summary.runId == runId && uiData.summary.name.equals("test"))) + assert(listener.activeQueryStatus.size() == 1) + assert(listener.activeQueryStatus.get(runId).name == "test") // handle query progress event val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) @@ -63,32 +53,28 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryProgress(processEvent) // result checking - val activeQuery = - queryStore.allQueryUIData.filter(_.summary.isActive).find(_.summary.runId == runId) - assert(activeQuery.isDefined) - assert(activeQuery.get.summary.isActive) - assert(activeQuery.get.recentProgress.length == 1) - assert(activeQuery.get.lastProgress.id == id) - assert(activeQuery.get.lastProgress.runId == runId) - assert(activeQuery.get.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") - assert(activeQuery.get.lastProgress.inputRowsPerSecond == 10.0) - assert(activeQuery.get.lastProgress.processedRowsPerSecond == 12.0) - assert(activeQuery.get.lastProgress.batchId == 2) - assert(activeQuery.get.lastProgress.prettyJson == """{"a":1}""") + val activeQuery = listener.activeQueryStatus.get(runId) + assert(activeQuery.isActive) + assert(activeQuery.recentProgress.length == 1) + assert(activeQuery.lastProgress.id == id) + assert(activeQuery.lastProgress.runId == runId) + assert(activeQuery.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") + assert(activeQuery.lastProgress.inputRowsPerSecond == 10.0) + assert(activeQuery.lastProgress.processedRowsPerSecond == 12.0) + assert(activeQuery.lastProgress.batchId == 2) + assert(activeQuery.lastProgress.prettyJson == """{"a":1}""") // handle terminate event val terminateEvent = new StreamingQueryListener.QueryTerminatedEvent(id, runId, None) listener.onQueryTerminated(terminateEvent) - assert(!queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.isActive) - assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId) - assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) + assert(!listener.inactiveQueryStatus.head.isActive) + assert(listener.inactiveQueryStatus.head.runId == runId) + assert(listener.inactiveQueryStatus.head.id == id) } test("same query start multiple times") { - val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) - val queryStore = new StreamingQueryStatusStore(kvStore) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) // handle first time start val id = UUID.randomUUID() @@ -108,106 +94,11 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent1) // result checking - assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) - assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).length == 1) - assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(_.summary.runId == runId1)) - assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => - uiData.summary.runId == runId1 && uiData.summary.id == id)) - assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId0) - assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) - } - - test("test small retained queries") { - val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) - val conf = spark.sparkContext.conf - conf.set(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES.key, "2") - val listener = new StreamingQueryStatusListener(conf, kvStore) - val queryStore = new StreamingQueryStatusStore(kvStore) - - def addNewQuery(): (UUID, UUID) = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 - format.setTimeZone(getTimeZone("UTC")) - val id = UUID.randomUUID() - val runId = UUID.randomUUID() - val startEvent = new StreamingQueryListener.QueryStartedEvent( - id, runId, "test1", format.format(new Date(System.currentTimeMillis()))) - listener.onQueryStarted(startEvent) - (id, runId) - } - - def checkInactiveQueryStatus(numInactives: Int, targetInactives: Seq[UUID]): Unit = { - eventually(timeout(10.seconds)) { - val inactiveQueries = queryStore.allQueryUIData.filter(!_.summary.isActive) - assert(inactiveQueries.size == numInactives) - assert(inactiveQueries.map(_.summary.id).toSet == targetInactives.toSet) - } - } - - val (id1, runId1) = addNewQuery() - val (id2, runId2) = addNewQuery() - val (id3, runId3) = addNewQuery() - assert(queryStore.allQueryUIData.count(!_.summary.isActive) == 0) - - val terminateEvent1 = new StreamingQueryListener.QueryTerminatedEvent(id1, runId1, None) - listener.onQueryTerminated(terminateEvent1) - checkInactiveQueryStatus(1, Seq(id1)) - val terminateEvent2 = new StreamingQueryListener.QueryTerminatedEvent(id2, runId2, None) - listener.onQueryTerminated(terminateEvent2) - checkInactiveQueryStatus(2, Seq(id1, id2)) - val terminateEvent3 = new StreamingQueryListener.QueryTerminatedEvent(id3, runId3, None) - listener.onQueryTerminated(terminateEvent3) - checkInactiveQueryStatus(2, Seq(id2, id3)) - } - - test("test small retained progress") { - val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) - val conf = spark.sparkContext.conf - conf.set(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES.key, "5") - val listener = new StreamingQueryStatusListener(conf, kvStore) - val queryStore = new StreamingQueryStatusStore(kvStore) - - val id = UUID.randomUUID() - val runId = UUID.randomUUID() - val startEvent = new StreamingQueryListener.QueryStartedEvent( - id, runId, "test", "2016-12-05T20:54:20.827Z") - listener.onQueryStarted(startEvent) - - var batchId: Int = 0 - - def addQueryProgress(): Unit = { - val progress = mockProgressData(id, runId) - val processEvent = new streaming.StreamingQueryListener.QueryProgressEvent(progress) - listener.onQueryProgress(processEvent) - } - - def mockProgressData(id: UUID, runId: UUID): StreamingQueryProgress = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 - format.setTimeZone(getTimeZone("UTC")) - - val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) - when(progress.id).thenReturn(id) - when(progress.runId).thenReturn(runId) - when(progress.timestamp).thenReturn(format.format(new Date(System.currentTimeMillis()))) - when(progress.inputRowsPerSecond).thenReturn(10.0) - when(progress.processedRowsPerSecond).thenReturn(12.0) - when(progress.batchId).thenReturn(batchId) - when(progress.prettyJson).thenReturn("""{"a":1}""") - - batchId += 1 - progress - } - - def checkQueryProcessData(targetNum: Int): Unit = { - eventually(timeout(10.seconds)) { - assert(queryStore.getQueryProgressData(runId).size == targetNum) - } - } - - Array.tabulate(4) { _ => addQueryProgress() } - checkQueryProcessData(4) - addQueryProgress() - checkQueryProcessData(5) - addQueryProgress() - checkQueryProcessData(5) + assert(listener.activeQueryStatus.size() == 1) + assert(listener.inactiveQueryStatus.length == 1) + assert(listener.activeQueryStatus.containsKey(runId1)) + assert(listener.activeQueryStatus.get(runId1).id == id) + assert(listener.inactiveQueryStatus.head.runId == runId0) + assert(listener.inactiveQueryStatus.head.id == id) } } From 50ee6f7f91a8dbcec1118b6352e5ebcb13c8c55a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 7 Dec 2020 15:47:30 +0900 Subject: [PATCH 2/3] Revert "Revert "[SPARK-31953][SS] Add Spark Structured Streaming History Server Support"" This reverts commit 11c024165e7d0ccc5b5c9f23e9406f8921528d2b. --- dev/.rat-excludes | 1 + ...apache.spark.status.AppHistoryServerPlugin | 1 + .../streaming/StreamingQueryListenerBus.scala | 26 ++- .../StreamingQueryHistoryServerPlugin.scala | 43 +++++ .../ui/StreamingQueryStatusStore.scala | 53 ++++++ .../spark/sql/internal/SharedState.scala | 8 +- .../sql/streaming/StreamingQueryManager.scala | 3 +- .../sql/streaming/ui/StreamingQueryPage.scala | 44 ++--- .../ui/StreamingQueryStatisticsPage.scala | 27 +-- .../ui/StreamingQueryStatusListener.scala | 166 +++++++++++------- .../sql/streaming/ui/StreamingQueryTab.scala | 3 +- .../spark/sql/streaming/ui/UIUtils.scala | 12 +- .../spark-events/local-1596020211915 | 160 +++++++++++++++++ .../apache/spark/deploy/history/Utils.scala | 40 +++++ .../ui/StreamingQueryHistorySuite.scala | 63 +++++++ .../ui/StreamingQueryPageSuite.scala | 42 +++-- .../StreamingQueryStatusListenerSuite.scala | 159 ++++++++++++++--- 17 files changed, 693 insertions(+), 158 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala create mode 100644 sql/core/src/test/resources/spark-events/local-1596020211915 create mode 100644 sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 7da330dfe1fb..167cf224f92c 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,6 +123,7 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties +local-1596020211915 app-20200706201101-0003 py.typed _metadata diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin index 0bba2f88b92a..6771eef52530 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -1 +1,2 @@ org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin +org.apache.spark.sql.execution.ui.StreamingQueryHistoryServerPlugin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 1b8d69ffb752..4b98acd16f6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -31,16 +31,21 @@ import org.apache.spark.util.ListenerBus * Spark listener bus, so that it can receive [[StreamingQueryListener.Event]]s and dispatch them * to StreamingQueryListeners. * - * Note that each bus and its registered listeners are associated with a single SparkSession + * Note 1: Each bus and its registered listeners are associated with a single SparkSession * and StreamingQueryManager. So this bus will dispatch events to registered listeners for only * those queries that were started in the associated SparkSession. + * + * Note 2: To rebuild Structured Streaming UI in SHS, this bus will be registered into + * [[org.apache.spark.scheduler.ReplayListenerBus]]. We check `sparkListenerBus` defined or not to + * determine how to process [[StreamingQueryListener.Event]]. If false, it means this bus is used to + * replay all streaming query event from eventLog. */ -class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) +class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) extends SparkListener with ListenerBus[StreamingQueryListener, StreamingQueryListener.Event] { import StreamingQueryListener._ - sparkListenerBus.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY) + sparkListenerBus.foreach(_.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY)) /** * RunIds of active queries whose events are supposed to be forwarded by this ListenerBus @@ -67,11 +72,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) event match { case s: QueryStartedEvent => activeQueryRunIds.synchronized { activeQueryRunIds += s.runId } - sparkListenerBus.post(s) + sparkListenerBus.foreach(bus => bus.post(s)) // post to local listeners to trigger callbacks postToAll(s) case _ => - sparkListenerBus.post(event) + sparkListenerBus.foreach(bus => bus.post(event)) } } @@ -95,7 +100,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) // synchronously and the ones attached to LiveListenerBus asynchronously. Therefore, // we need to ignore QueryStartedEvent if this method is called within SparkListenerBus // thread - if (!LiveListenerBus.withinListenerThread.value || !e.isInstanceOf[QueryStartedEvent]) { + // + // When loaded by Spark History Server, we should process all event coming from replay + // listener bus. + if (sparkListenerBus.isEmpty || !LiveListenerBus.withinListenerThread.value || + !e.isInstanceOf[QueryStartedEvent]) { postToAll(e) } case _ => @@ -110,7 +119,10 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) listener: StreamingQueryListener, event: StreamingQueryListener.Event): Unit = { def shouldReport(runId: UUID): Boolean = { - activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } + // When loaded by Spark History Server, we should process all event coming from replay + // listener bus. + sparkListenerBus.isEmpty || + activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } } event match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala new file mode 100644 index 000000000000..a127fa59b743 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala @@ -0,0 +1,43 @@ +/* + * 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.sql.execution.ui + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus +import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} +import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} +import org.apache.spark.ui.SparkUI + +class StreamingQueryHistoryServerPlugin extends AppHistoryServerPlugin { + + override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { + val listenerBus = new StreamingQueryListenerBus(None) + listenerBus.addListener(new StreamingQueryStatusListener(conf, store)) + Seq(listenerBus) + } + + override def setupUI(ui: SparkUI): Unit = { + val streamingQueryStatusStore = new StreamingQueryStatusStore(ui.store.store) + if (streamingQueryStatusStore.allQueryUIData.nonEmpty) { + new StreamingQueryTab(streamingQueryStatusStore, ui) + } + } + + override def displayOrder: Int = 1 +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala new file mode 100644 index 000000000000..9eb14a6a6306 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.execution.ui + +import java.util.UUID + +import org.apache.spark.sql.streaming.ui.{StreamingQueryData, StreamingQueryProgressWrapper, StreamingQueryUIData} +import org.apache.spark.status.KVUtils +import org.apache.spark.util.kvstore.KVStore + +/** + * Provides a view of a KVStore with methods that make it easy to query Streaming Query state. + * There's no state kept in this class, so it's ok to have multiple instances of it in an + * application. + */ +class StreamingQueryStatusStore(store: KVStore) { + + def allQueryUIData: Seq[StreamingQueryUIData] = { + val view = store.view(classOf[StreamingQueryData]).index("startTimestamp").first(0L) + KVUtils.viewToSeq(view, Int.MaxValue)(_ => true).map(makeUIData) + } + + // visible for test + private[sql] def getQueryProgressData(runId: UUID): Seq[StreamingQueryProgressWrapper] = { + val view = store.view(classOf[StreamingQueryProgressWrapper]) + .index("runId").first(runId.toString).last(runId.toString) + KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + } + + private def makeUIData(summary: StreamingQueryData): StreamingQueryUIData = { + val runId = summary.runId.toString + val view = store.view(classOf[StreamingQueryProgressWrapper]) + .index("runId").first(runId).last(runId) + val recentProgress = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + .map(_.progress).sortBy(_.timestamp).toArray + StreamingQueryUIData(summary, recentProgress) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 89aceacac600..ea430db9f030 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} +import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab, StreamingQueryStatusStore} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} import org.apache.spark.status.ElementTrackingStore @@ -111,9 +111,9 @@ private[sql] class SharedState( lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] = { sparkContext.ui.flatMap { ui => if (conf.get(STREAMING_UI_ENABLED)) { - val statusListener = new StreamingQueryStatusListener(conf) - new StreamingQueryTab(statusListener, ui) - Some(statusListener) + val kvStore = sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + new StreamingQueryTab(new StreamingQueryStatusStore(kvStore), ui) + Some(new StreamingQueryStatusListener(conf, kvStore)) } else { None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index ffdbe9d4e491..b66037d00919 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -49,7 +49,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) - private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) + private val listenerBus = + new StreamingQueryListenerBus(Some(sparkSession.sparkContext.listenerBus)) @GuardedBy("activeQueriesSharedLock") private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala index b98fdf16eef3..96e498991e1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -40,8 +40,8 @@ private[ui] class StreamingQueryPage(parent: StreamingQueryTab) } private def generateStreamingQueryTable(request: HttpServletRequest): Seq[Node] = { - val (activeQueries, inactiveQueries) = parent.statusListener.allQueryStatus - .partition(_.isActive) + val (activeQueries, inactiveQueries) = + parent.store.allQueryUIData.partition(_.summary.isActive) val content = mutable.ListBuffer[Node]() // show active queries table only if there is at least one active query @@ -176,7 +176,7 @@ class StreamingQueryPagedTable( val streamingQuery = query.streamingUIData val statisticsLink = "%s/%s/statistics?id=%s" .format(SparkUIUtils.prependBaseUri(request, parent.basePath), parent.prefix, - streamingQuery.runId) + streamingQuery.summary.runId) def details(detail: Any): Seq[Node] = { if (isActive) { @@ -194,14 +194,14 @@ class StreamingQueryPagedTable( {UIUtils.getQueryName(streamingQuery)} {UIUtils.getQueryStatus(streamingQuery)} - {streamingQuery.id} - {streamingQuery.runId} - {SparkUIUtils.formatDate(streamingQuery.startTimestamp)} + {streamingQuery.summary.id} + {streamingQuery.summary.runId} + {SparkUIUtils.formatDate(streamingQuery.summary.startTimestamp)} {SparkUIUtils.formatDurationVerbose(query.duration)} {withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} - {details(streamingQuery.exception.getOrElse("-"))} + {details(streamingQuery.summary.exception.getOrElse("-"))} } } @@ -222,32 +222,32 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St override def sliceData(from: Int, to: Int): Seq[StructuredStreamingRow] = data.slice(from, to) - private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = { + private def streamingRow(uiData: StreamingQueryUIData): StructuredStreamingRow = { val duration = if (isActive) { - System.currentTimeMillis() - query.startTimestamp + System.currentTimeMillis() - uiData.summary.startTimestamp } else { - withNoProgress(query, { - val endTimeMs = query.lastProgress.timestamp - parseProgressTimestamp(endTimeMs) - query.startTimestamp + withNoProgress(uiData, { + val endTimeMs = uiData.lastProgress.timestamp + parseProgressTimestamp(endTimeMs) - uiData.summary.startTimestamp }, 0) } - val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / - query.recentProgress.length) + val avgInput = (uiData.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / + uiData.recentProgress.length) - val avgProcess = (query.recentProgress.map(p => - withNumberInvalid(p.processedRowsPerSecond)).sum / query.recentProgress.length) + val avgProcess = (uiData.recentProgress.map(p => + withNumberInvalid(p.processedRowsPerSecond)).sum / uiData.recentProgress.length) - StructuredStreamingRow(duration, avgInput, avgProcess, query) + StructuredStreamingRow(duration, avgInput, avgProcess, uiData) } private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = { val ordering: Ordering[StructuredStreamingRow] = sortColumn match { - case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData)) - case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData)) - case "ID" => Ordering.by(_.streamingUIData.id) - case "Run ID" => Ordering.by(_.streamingUIData.runId) - case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp) + case "Name" => Ordering.by(row => UIUtils.getQueryName(row.streamingUIData)) + case "Status" => Ordering.by(row => UIUtils.getQueryStatus(row.streamingUIData)) + case "ID" => Ordering.by(_.streamingUIData.summary.id) + case "Run ID" => Ordering.by(_.streamingUIData.summary.runId) + case "Start Time" => Ordering.by(_.streamingUIData.summary.startTimestamp) case "Duration" => Ordering.by(_.duration) case "Avg Input /sec" => Ordering.by(_.avgInput) case "Avg Process /sec" => Ordering.by(_.avgProcess) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala index 24709ba470cd..97691d9d7e82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala @@ -58,8 +58,8 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val query = parent.statusListener.allQueryStatus.find { case q => - q.runId.equals(UUID.fromString(parameterId)) + val query = parent.store.allQueryUIData.find { uiData => + uiData.summary.runId.equals(UUID.fromString(parameterId)) }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming query $parameterId")) val resources = generateLoadResources(request) @@ -109,34 +109,35 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) } - def generateBasicInfo(query: StreamingQueryUIData): Seq[Node] = { - val duration = if (query.isActive) { - SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp) + def generateBasicInfo(uiData: StreamingQueryUIData): Seq[Node] = { + val duration = if (uiData.summary.isActive) { + val durationMs = System.currentTimeMillis() - uiData.summary.startTimestamp + SparkUIUtils.formatDurationVerbose(durationMs) } else { - withNoProgress(query, { - val end = query.lastProgress.timestamp - val start = query.recentProgress.head.timestamp + withNoProgress(uiData, { + val end = uiData.lastProgress.timestamp + val start = uiData.recentProgress.head.timestamp SparkUIUtils.formatDurationVerbose( parseProgressTimestamp(end) - parseProgressTimestamp(start)) }, "-") } - val name = UIUtils.getQueryName(query) - val numBatches = withNoProgress(query, { query.lastProgress.batchId + 1L }, 0) + val name = UIUtils.getQueryName(uiData) + val numBatches = withNoProgress(uiData, { uiData.lastProgress.batchId + 1L }, 0)
Running batches for {duration} since - {SparkUIUtils.formatDate(query.startTimestamp)} + {SparkUIUtils.formatDate(uiData.summary.startTimestamp)} ({numBatches} completed batches)

Name: {name}
-
Id: {query.id}
-
RunId: {query.runId}
+
Id: {uiData.summary.id}
+
RunId: {uiData.summary.runId}

} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala index e331083b3002..fdd375434410 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala @@ -20,102 +20,144 @@ package org.apache.spark.sql.streaming.ui import java.util.UUID import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConverters._ import scala.collection.mutable +import com.fasterxml.jackson.annotation.JsonIgnore + import org.apache.spark.SparkConf import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress} +import org.apache.spark.sql.streaming.ui.StreamingQueryProgressWrapper._ import org.apache.spark.sql.streaming.ui.UIUtils.parseProgressTimestamp +import org.apache.spark.status.{ElementTrackingStore, KVUtils} +import org.apache.spark.status.KVUtils.KVIndexParam +import org.apache.spark.util.kvstore.KVIndex /** * A customized StreamingQueryListener used in structured streaming UI, which contains all * UI data for both active and inactive query. - * TODO: Add support for history server. */ -private[sql] class StreamingQueryStatusListener(conf: SparkConf) extends StreamingQueryListener { - - /** - * We use runId as the key here instead of id in active query status map, - * because the runId is unique for every started query, even it its a restart. - */ - private[ui] val activeQueryStatus = new ConcurrentHashMap[UUID, StreamingQueryUIData]() - private[ui] val inactiveQueryStatus = new mutable.Queue[StreamingQueryUIData]() +private[sql] class StreamingQueryStatusListener( + conf: SparkConf, + store: ElementTrackingStore) extends StreamingQueryListener { private val streamingProgressRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES) private val inactiveQueryStatusRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES) + store.addTrigger(classOf[StreamingQueryData], inactiveQueryStatusRetention) { count => + cleanupInactiveQueries(count) + } + + // Events from the same query run will never be processed concurrently, so it's safe to + // access `progressIds` without any protection. + private val queryToProgress = new ConcurrentHashMap[UUID, mutable.Queue[String]]() + + private def cleanupInactiveQueries(count: Long): Unit = { + val view = store.view(classOf[StreamingQueryData]).index("active").first(false).last(false) + val inactiveQueries = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + val numInactiveQueries = inactiveQueries.size + if (numInactiveQueries <= inactiveQueryStatusRetention) { + return + } + val toDelete = inactiveQueries.sortBy(_.endTimestamp.get) + .take(numInactiveQueries - inactiveQueryStatusRetention) + val runIds = toDelete.map { e => + store.delete(e.getClass, e.runId) + e.runId.toString + } + // Delete wrappers in one pass, as deleting them for each summary is slow + store.removeAllByIndexValues(classOf[StreamingQueryProgressWrapper], "runId", runIds) + } + override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { val startTimestamp = parseProgressTimestamp(event.timestamp) - activeQueryStatus.putIfAbsent(event.runId, - new StreamingQueryUIData(event.name, event.id, event.runId, startTimestamp)) + store.write(new StreamingQueryData( + event.name, + event.id, + event.runId, + isActive = true, + None, + startTimestamp + ), checkTriggers = true) } override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { - val batchTimestamp = parseProgressTimestamp(event.progress.timestamp) - val queryStatus = activeQueryStatus.getOrDefault( - event.progress.runId, - new StreamingQueryUIData(event.progress.name, event.progress.id, event.progress.runId, - batchTimestamp)) - queryStatus.updateProcess(event.progress, streamingProgressRetention) - } - - override def onQueryTerminated( - event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { - val queryStatus = activeQueryStatus.remove(event.runId) - if (queryStatus != null) { - queryStatus.queryTerminated(event) - inactiveQueryStatus += queryStatus - while (inactiveQueryStatus.length >= inactiveQueryStatusRetention) { - inactiveQueryStatus.dequeue() - } + val runId = event.progress.runId + val batchId = event.progress.batchId + val timestamp = event.progress.timestamp + if (!queryToProgress.containsKey(runId)) { + queryToProgress.put(runId, mutable.Queue.empty[String]) + } + val progressIds = queryToProgress.get(runId) + progressIds.enqueue(getUniqueId(runId, batchId, timestamp)) + store.write(new StreamingQueryProgressWrapper(event.progress)) + while (progressIds.length > streamingProgressRetention) { + val uniqueId = progressIds.dequeue + store.delete(classOf[StreamingQueryProgressWrapper], uniqueId) } } - def allQueryStatus: Seq[StreamingQueryUIData] = synchronized { - activeQueryStatus.values().asScala.toSeq ++ inactiveQueryStatus + override def onQueryTerminated( + event: StreamingQueryListener.QueryTerminatedEvent): Unit = { + val querySummary = store.read(classOf[StreamingQueryData], event.runId) + val curTime = System.currentTimeMillis() + store.write(new StreamingQueryData( + querySummary.name, + querySummary.id, + querySummary.runId, + isActive = false, + querySummary.exception, + querySummary.startTimestamp, + Some(curTime) + ), checkTriggers = true) + queryToProgress.remove(event.runId) } } +private[sql] class StreamingQueryData( + val name: String, + val id: UUID, + @KVIndexParam val runId: UUID, + @KVIndexParam("active") val isActive: Boolean, + val exception: Option[String], + @KVIndexParam("startTimestamp") val startTimestamp: Long, + val endTimestamp: Option[Long] = None) + /** * This class contains all message related to UI display, each instance corresponds to a single * [[org.apache.spark.sql.streaming.StreamingQuery]]. */ -private[ui] class StreamingQueryUIData( - val name: String, - val id: UUID, - val runId: UUID, - val startTimestamp: Long) { - - /** Holds the most recent query progress updates. */ - private val progressBuffer = new mutable.Queue[StreamingQueryProgress]() - - private var _isActive = true - private var _exception: Option[String] = None - - def isActive: Boolean = synchronized { _isActive } - - def exception: Option[String] = synchronized { _exception } - - def queryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { - _isActive = false - _exception = event.exception - } - - def updateProcess( - newProgress: StreamingQueryProgress, retentionNum: Int): Unit = progressBuffer.synchronized { - progressBuffer += newProgress - while (progressBuffer.length >= retentionNum) { - progressBuffer.dequeue() +private[sql] case class StreamingQueryUIData( + summary: StreamingQueryData, + recentProgress: Array[StreamingQueryProgress]) { + + def lastProgress: StreamingQueryProgress = { + if (recentProgress.nonEmpty) { + recentProgress.last + } else { + null } } +} - def recentProgress: Array[StreamingQueryProgress] = progressBuffer.synchronized { - progressBuffer.toArray - } +private[sql] class StreamingQueryProgressWrapper(val progress: StreamingQueryProgress) { + @JsonIgnore @KVIndex + private val uniqueId: String = getUniqueId(progress.runId, progress.batchId, progress.timestamp) - def lastProgress: StreamingQueryProgress = progressBuffer.synchronized { - progressBuffer.lastOption.orNull + @JsonIgnore @KVIndex("runId") + private def runIdIndex: String = progress.runId.toString +} + +private[sql] object StreamingQueryProgressWrapper { + /** + * Adding `timestamp` into unique id to support reporting `empty` query progress + * in which no data comes but with the same batchId. + */ + def getUniqueId( + runId: UUID, + batchId: Long, + timestamp: String): String = { + s"${runId}_${batchId}_$timestamp" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala index bb097ffc0691..65cad8f06cc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.streaming.ui import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.ui.{SparkUI, SparkUITab} private[sql] class StreamingQueryTab( - val statusListener: StreamingQueryStatusListener, + val store: StreamingQueryStatusStore, sparkUI: SparkUI) extends SparkUITab(sparkUI, "StreamingQuery") with Logging { override val name = "Structured Streaming" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala index 1f7e65dede17..88a110fa9a32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala @@ -46,19 +46,19 @@ private[ui] object UIUtils { } } - def getQueryName(query: StreamingQueryUIData): String = { - if (query.name == null || query.name.isEmpty) { + def getQueryName(uiData: StreamingQueryUIData): String = { + if (uiData.summary.name == null || uiData.summary.name.isEmpty) { "" } else { - query.name + uiData.summary.name } } - def getQueryStatus(query: StreamingQueryUIData): String = { - if (query.isActive) { + def getQueryStatus(uiData: StreamingQueryUIData): String = { + if (uiData.summary.isActive) { "RUNNING" } else { - query.exception.map(_ => "FAILED").getOrElse("FINISHED") + uiData.summary.exception.map(_ => "FAILED").getOrElse("FINISHED") } } diff --git a/sql/core/src/test/resources/spark-events/local-1596020211915 b/sql/core/src/test/resources/spark-events/local-1596020211915 new file mode 100644 index 000000000000..ff34bbc16ef3 --- /dev/null +++ b/sql/core/src/test/resources/spark-events/local-1596020211915 @@ -0,0 +1,160 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1596020212090,"Executor ID":"driver","Executor Info":{"Host":"iZbp19vpr16ix621sdw476Z","Total Cores":4,"Log Urls":{},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Port":39845},"Maximum Memory":384093388,"Timestamp":1596020212109,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","Java Version":"1.8.0_252 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.driver.port":"46309","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","spark.app.name":"StructuredKafkaWordCount","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","fs.s3.block.size":"67108864","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","mapred.child.java.opts":"-Xmx200m","hadoop.common.configuration.version":"0.23.0","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","mapreduce.task.profile.reduces":"0-2","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.nodemanager.webapp.cross-origin.enabled":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.s3a.fast.upload.active.blocks":"4","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","ftp.blocksize":"67108864","yarn.nodemanager.log-container-debug-info.enabled":"false","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.amrmproxy.enable":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"false","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","io.map.index.interval":"128","mapreduce.job.counters.max":"120","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","s3native.bytes-per-checksum":"512","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","yarn.node-labels.fs-store.retry-policy-spec":"2000, 500","hadoop.security.groups.cache.background.reload":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","hadoop.http.authentication.type":"simple","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","yarn.nodemanager.resource-monitor.interval-ms":"3000","mapreduce.shuffle.listen.queue.size":"128","mapreduce.map.cpu.vcores":"1","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.app.mapreduce.client.job.max-retries":"0","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.timeline-service.client.internal-timers-ttl-secs":"420","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"1024","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","fs.s3n.multipart.uploads.block.size":"67108864","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"1024","s3native.client-write-packet-size":"65536","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","yarn.sharedcache.nm.uploader.thread-count":"20","mapreduce.shuffle.ssl.enabled":"false","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"json","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","yarn.resourcemanager.zk-num-retries":"1000","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8048","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME","yarn.sharedcache.nested-level":"3","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false"},"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":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/amd64","user.dir":"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.252-b09","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_252-b09","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"Asia/Shanghai","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-1127.10.1.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","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":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[*] --conf spark.eventLog.dir=/tmp/spark-history --conf spark.eventLog.enabled=true --conf spark.sql.shuffle.partitions=2 --class org.apache.spark.examples.sql.streaming.StructuredKafkaWordCount ./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar 192.168.130.97:9092 subscribe test5","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","java.version":"1.8.0_252","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-graphite-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-server-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/pyrolite-4.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/conf/":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-smart-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/objenesis-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-auth-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsp-api-2.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-codec-1.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/protobuf-java-2.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/transaction-api-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/gson-2.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libfb303-0.9.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-cli-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-library-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-api-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okhttp-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/derby-10.12.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-collections-3.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpcore-4.4.12.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-beanutils-1.9.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-crypto-1.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-ast_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/lz4-java-1.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-format-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-column-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-logging-1.1.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/audience-annotations-0.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-jdbc-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive-thriftserver_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-cli-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javolution-5.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JLargeArrays-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-api-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-dbcp-1.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/threeten-extra-1.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-io-2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-json-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libthrift-0.12.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/compress-lzf-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jmx-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.inject-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-recipes-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-api-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/oro-2.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-memory-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jpam-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/velocity-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-core-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-databind-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-text-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-client-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-ipc-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-util-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/core-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/joda-time-2.10.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-encoding-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-llap-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/paranamer-2.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/activation-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-framework-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compress-1.8.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/RoaringBitmap-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ivy-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-httpclient-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-yarn_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-common-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zstd-jni-1.4.5-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/snappy-java-1.1.7.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool-1.5.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-core_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpclient-4.5.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-util-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aircompressor-0.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/leveldbjni-all-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-hk2-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jta-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-net-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-core-4.1.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-web-proxy-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-mapreduce-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xz-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.inject-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-compiler-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jvm-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-shims-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-api-2.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/janino-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-app-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-utils-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JTransforms-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kafka-clients-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-servlet-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okio-1.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-annotations-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-math3-3.4.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-scalap_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/accessors-smart-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guava-14.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shims-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ST4-4.0.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill-java-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-shuffle-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stream-2.9.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-configuration-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jodd-core-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool2-2.6.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xmlenc-0.52.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-token-provider-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-jackson_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compiler-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jline-2.14.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kryo-shaded-4.0.2.jar":"System Classpath","spark://iZbp19vpr16ix621sdw476Z:46309/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar":"Added By User","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-common-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-client-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-xc-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zookeeper-3.4.14.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/univocity-parsers-2.8.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-digester-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arpack_combined_all-0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-annotations-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-locator-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-core-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-reflect-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/super-csv-2.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-jobclient-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-exec-2.3.7-core.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/opencsv-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-storage-api-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/HikariCP-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-metastore-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/minlog-1.3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-format-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsr305-3.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang-2.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang3-3.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javassist-3.25.0-GA.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr-runtime-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/log4j-1.2.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-beeline-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill_2.12-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jdo-api-3.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-core-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/py4j-0.10.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-serde-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-hdfs-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcip-annotations-1.0-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-jackson-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/netty-all-4.1.47.Final.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-vector-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/machinist_2.12-0.6.8.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"StructuredKafkaWordCount","App ID":"local-1596020211915","Timestamp":1596020210919,"User":"root"} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryStartedEvent","id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:55.947Z"} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220179} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#218]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220258} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1596020221633,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0,1],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222649,"Failed":false,"Killed":false,"Accumulables":[{"ID":21,"Name":"shuffle write time","Update":"9599308","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Update":"296","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Update":"200","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"time in aggregation build","Update":"190","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"duration","Update":"336","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Update":9599308,"Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":17,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Update":2630,"Value":2630,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorCpuTime","Update":466139164,"Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Update":503,"Value":503,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Update":301869581,"Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Update":361,"Value":361,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":361,"Executor Deserialize CPU Time":301869581,"Executor Run Time":503,"Executor CPU Time":466139164,"Peak Execution Memory":524288,"Result Size":2630,"JVM GC Time":17,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":9599308,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Completion Time":1596020222661,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"time in aggregation build","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Value":2630,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":106,"Name":"internal.metrics.executorCpuTime","Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Value":361,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"duration","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Value":503,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":108,"Name":"internal.metrics.jvmGCTime","Value":17,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"shuffle write time","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222954,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"50","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"64","Value":"64","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"208","Value":"208","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5354,"Value":5354,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":93367533,"Value":93367533,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":203,"Value":203,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10308753,"Value":10308753,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":10308753,"Executor Run Time":203,"Executor CPU Time":93367533,"Peak Execution Memory":524288,"Result Size":5354,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222965,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"33","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":45,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":53,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"31","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"424","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":50,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"568","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5574,"Value":10928,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":91355172,"Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":205,"Value":408,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":21029530,"Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":34,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":34,"Executor Deserialize CPU Time":21029530,"Executor Run Time":205,"Executor CPU Time":91355172,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Completion Time":1596020222967,"Accumulables":[{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":50,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"duration","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"estimated size of state only on current version","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":130,"Name":"internal.metrics.executorRunTime","Value":408,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Value":10928,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1596020222973,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#46, count#47]\nArguments: [value#46, count#47]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#46, count#47]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":153,"metricType":"sum"}]},"time":1596020223028} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1596020223062} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1596020223069} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1596020223069} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:56.015Z","batchId":0,"batchDuration":7110,"durationMs":{"triggerExecution":7109,"queryPlanning":439,"getBatch":21,"latestOffset":3524,"addBatch":3011,"walCommit":35},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":776,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":488,"loadedMapCacheHitCount":0,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":null,"endOffset":"{\"test5\":{\"0\":48279}}","numInputRows":3,"inputRowsPerSecond":"NaN","processedRowsPerSecond":0.42194092827004215}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223333} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#449]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223382} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1596020223482,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2,3],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223601,"Failed":false,"Killed":false,"Accumulables":[{"ID":178,"Name":"shuffle write time","Update":"837580","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":226,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"time in aggregation build","Update":"74","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":234,"Name":"time in aggregation build","Update":"68","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":237,"Name":"number of output rows","Update":"363","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":282,"Name":"internal.metrics.input.recordsRead","Update":363,"Value":363,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Update":837580,"Value":837580,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.executorCpuTime","Update":95945587,"Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.executorRunTime","Update":96,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7437557,"Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7437557,"Executor Run Time":96,"Executor CPU Time":95945587,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":169,"Shuffle Write Time":837580,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":363},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Completion Time":1596020223603,"Accumulables":[{"ID":227,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":262,"Name":"internal.metrics.executorRunTime","Value":96,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Value":837580,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"time in aggregation build","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"shuffle write time","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"time in aggregation build","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":264,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":282,"Name":"internal.metrics.input.recordsRead","Value":363,"Internal":true,"Count Failed Values":true},{"ID":237,"Name":"number of output rows","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":263,"Name":"internal.metrics.executorCpuTime","Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":169,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223717,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"376","Value":"376","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":22954307,"Value":22954307,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":77,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6627382,"Value":6627382,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6627382,"Executor Run Time":77,"Executor CPU Time":22954307,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223720,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":206,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"18","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"30","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"840","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"19","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":175,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":25907369,"Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":82,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7573630,"Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":13,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7573630,"Executor Run Time":82,"Executor CPU Time":25907369,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":169,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Completion Time":1596020223724,"Accumulables":[{"ID":218,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":175,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":169,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"count of cache hit on states cache in provider","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":201,"Name":"duration","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":287,"Name":"internal.metrics.executorRunTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1596020223725,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#60, count#61]\nArguments: [value#60, count#61]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#60, count#61]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":310,"metricType":"sum"}]},"time":1596020223752} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1596020223761} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1596020223762} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1596020223762} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.168Z","batchId":1,"batchDuration":622,"durationMs":{"triggerExecution":622,"queryPlanning":47,"getBatch":0,"latestOffset":7,"addBatch":478,"walCommit":59},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1216,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":4,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48279}}","endOffset":"{\"test5\":{\"0\":48642}}","numInputRows":363,"inputRowsPerSecond":50.74793792814204,"processedRowsPerSecond":583.6012861736334}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223909} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#680]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224006} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1596020224100,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[5,4],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224174,"Failed":false,"Killed":false,"Accumulables":[{"ID":335,"Name":"shuffle write time","Update":"686296","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Update":"39","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":384,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":386,"Name":"time in aggregation build","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":388,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":391,"Name":"time in aggregation build","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Update":"40","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":394,"Name":"number of output rows","Update":"63","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Update":686296,"Value":686296,"Internal":true,"Count Failed Values":true},{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":420,"Name":"internal.metrics.executorCpuTime","Update":33390843,"Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4867521,"Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":4867521,"Executor Run Time":49,"Executor CPU Time":33390843,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":686296,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":63},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Completion Time":1596020224175,"Accumulables":[{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":391,"Name":"time in aggregation build","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":394,"Name":"number of output rows","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":388,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":420,"Name":"internal.metrics.executorCpuTime","Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":384,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Value":686296,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Value":49,"Internal":true,"Count Failed Values":true},{"ID":386,"Name":"time in aggregation build","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":335,"Name":"shuffle write time","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224256,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":17230622,"Value":17230622,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5948051,"Value":5948051,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5948051,"Executor Run Time":56,"Executor CPU Time":17230622,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224257,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"21","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"18","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":364,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"22","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":332,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":23808555,"Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6247106,"Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6247106,"Executor Run Time":56,"Executor CPU Time":23808555,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Completion Time":1596020224259,"Accumulables":[{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"count of cache hit on states cache in provider","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":358,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":369,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":360,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":372,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":444,"Name":"internal.metrics.executorRunTime","Value":112,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1596020224259,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#74, count#75]\nArguments: [value#74, count#75]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#74, count#75]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":467,"metricType":"sum"}]},"time":1596020224278} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1596020224287} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1596020224287} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1596020224288} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.793Z","batchId":2,"batchDuration":522,"durationMs":{"triggerExecution":522,"queryPlanning":41,"getBatch":1,"latestOffset":3,"addBatch":421,"walCommit":27},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":8,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48642}}","endOffset":"{\"test5\":{\"0\":48705}}","numInputRows":63,"inputRowsPerSecond":100.8,"processedRowsPerSecond":120.6896551724138}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":9,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224419} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":10,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#911]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224452} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1596020224533,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[6,7],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224581,"Failed":false,"Killed":false,"Accumulables":[{"ID":492,"Name":"shuffle write time","Update":"643278","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":490,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":540,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":543,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":548,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":550,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":551,"Name":"number of output rows","Update":"52","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":596,"Name":"internal.metrics.input.recordsRead","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Update":643278,"Value":643278,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":578,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":577,"Name":"internal.metrics.executorCpuTime","Update":29099071,"Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":576,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3091128,"Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3091128,"Executor Run Time":29,"Executor CPU Time":29099071,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":643278,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":52},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Completion Time":1596020224582,"Accumulables":[{"ID":550,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":577,"Name":"internal.metrics.executorCpuTime","Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":490,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":576,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":540,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Value":643278,"Internal":true,"Count Failed Values":true},{"ID":543,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":492,"Name":"shuffle write time","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":548,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":578,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":596,"Name":"internal.metrics.input.recordsRead","Value":52,"Internal":true,"Count Failed Values":true},{"ID":551,"Name":"number of output rows","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224670,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"27","Value":"27","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":19967906,"Value":19967906,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4899567,"Value":4899567,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4899567,"Executor Run Time":62,"Executor CPU Time":19967906,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224687,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":516,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"26","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":521,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":523,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":484,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":488,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":22402538,"Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":79,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4671511,"Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4671511,"Executor Run Time":79,"Executor CPU Time":22402538,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Completion Time":1596020224688,"Accumulables":[{"ID":523,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":534,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":521,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":515,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":536,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":526,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Value":141,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1596020224689,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":11,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#88, count#89]\nArguments: [value#88, count#89]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#88, count#89]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":624,"metricType":"sum"}]},"time":1596020224709} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":11,"time":1596020224713} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":10,"time":1596020224714} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":9,"time":1596020224714} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.317Z","batchId":3,"batchDuration":415,"durationMs":{"triggerExecution":415,"queryPlanning":38,"getBatch":1,"latestOffset":3,"addBatch":332,"walCommit":21},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":12,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48705}}","endOffset":"{\"test5\":{\"0\":48757}}","numInputRows":52,"inputRowsPerSecond":99.23664122137404,"processedRowsPerSecond":125.30120481927712}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":12,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1066]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224817} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":13,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224849} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1596020224928,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[9,8],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224979,"Failed":false,"Killed":false,"Accumulables":[{"ID":649,"Name":"shuffle write time","Update":"572754","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":648,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":696,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":697,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":698,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":700,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":705,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":707,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":708,"Name":"number of output rows","Update":"42","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Update":42,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Update":572754,"Value":572754,"Internal":true,"Count Failed Values":true},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":735,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":734,"Name":"internal.metrics.executorCpuTime","Update":27800373,"Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":733,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4768103,"Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4768103,"Executor Run Time":28,"Executor CPU Time":27800373,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":572754,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":42},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Completion Time":1596020224979,"Accumulables":[{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":696,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":705,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":708,"Name":"number of output rows","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Value":42,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":698,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":734,"Name":"internal.metrics.executorCpuTime","Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":707,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":733,"Name":"internal.metrics.executorRunTime","Value":28,"Internal":true,"Count Failed Values":true},{"ID":697,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Value":572754,"Internal":true,"Count Failed Values":true},{"ID":700,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":649,"Name":"shuffle write time","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225056,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":678,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":690,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":641,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":644,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":646,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":19548688,"Value":19548688,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5622533,"Value":5622533,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":5,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5622533,"Executor Run Time":52,"Executor CPU Time":19548688,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225058,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"35","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":16813539,"Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":55,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4322992,"Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":9,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4322992,"Executor Run Time":55,"Executor CPU Time":16813539,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Completion Time":1596020225059,"Accumulables":[{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":678,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":759,"Name":"internal.metrics.executorCpuTime","Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"count of cache hit on states cache in provider","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":690,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":672,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.executorRunTime","Value":107,"Internal":true,"Count Failed Values":true},{"ID":644,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1596020225059,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":14,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#102, count#103]\nArguments: [value#102, count#103]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#102, count#103]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":781,"metricType":"sum"}]},"time":1596020225079} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":14,"time":1596020225087} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":13,"time":1596020225087} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":12,"time":1596020225087} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.734Z","batchId":4,"batchDuration":387,"durationMs":{"triggerExecution":387,"queryPlanning":30,"getBatch":1,"latestOffset":3,"addBatch":306,"walCommit":12},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":16,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48757}}","endOffset":"{\"test5\":{\"0\":48799}}","numInputRows":42,"inputRowsPerSecond":100.71942446043165,"processedRowsPerSecond":108.52713178294573}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":15,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1297]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225211} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":16,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225270} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1596020225342,"Stage Infos":[{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[10,11],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225400,"Failed":false,"Killed":false,"Accumulables":[{"ID":806,"Name":"shuffle write time","Update":"530930","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":855,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":857,"Name":"time in aggregation build","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":864,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":865,"Name":"number of output rows","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":910,"Name":"internal.metrics.input.recordsRead","Update":38,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Update":530930,"Value":530930,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":891,"Name":"internal.metrics.executorCpuTime","Update":22440089,"Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6808170,"Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6808170,"Executor Run Time":29,"Executor CPU Time":22440089,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":530930,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":38},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Completion Time":1596020225401,"Accumulables":[{"ID":855,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":891,"Name":"internal.metrics.executorCpuTime","Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":864,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Value":530930,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":857,"Name":"time in aggregation build","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":806,"Name":"shuffle write time","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":910,"Name":"internal.metrics.input.recordsRead","Value":38,"Internal":true,"Count Failed Values":true},{"ID":865,"Name":"number of output rows","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225498,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":830,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"37","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":802,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":17945299,"Value":17945299,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":68,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3451032,"Value":3451032,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3451032,"Executor Run Time":68,"Executor CPU Time":17945299,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225509,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"50","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":15599091,"Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":84,"Value":152,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4357806,"Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4357806,"Executor Run Time":84,"Executor CPU Time":15599091,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Completion Time":1596020225514,"Accumulables":[{"ID":846,"Name":"duration","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":830,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":833,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":844,"Name":"count of cache hit on states cache in provider","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":916,"Name":"internal.metrics.executorCpuTime","Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":829,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":840,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":915,"Name":"internal.metrics.executorRunTime","Value":152,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1596020225514,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":17,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#116, count#117]\nArguments: [value#116, count#117]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#116, count#117]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":938,"metricType":"sum"}]},"time":1596020225536} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":17,"time":1596020225541} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":16,"time":1596020225542} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":15,"time":1596020225542} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.123Z","batchId":5,"batchDuration":437,"durationMs":{"triggerExecution":437,"queryPlanning":35,"getBatch":1,"latestOffset":3,"addBatch":361,"walCommit":18},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":20,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48799}}","endOffset":"{\"test5\":{\"0\":48837}}","numInputRows":38,"inputRowsPerSecond":97.68637532133675,"processedRowsPerSecond":86.95652173913044}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":18,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1528]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225657} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":19,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225687} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1596020225759,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[12,13],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225796,"Failed":false,"Killed":false,"Accumulables":[{"ID":963,"Name":"shuffle write time","Update":"543836","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1010,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1012,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1014,"Name":"time in aggregation build","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1016,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1019,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1022,"Name":"number of output rows","Update":"44","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Update":44,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Update":543836,"Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1049,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Update":23733439,"Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3714406,"Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3714406,"Executor Run Time":23,"Executor CPU Time":23733439,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543836,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":44},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Completion Time":1596020225797,"Accumulables":[{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1010,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1019,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Value":44,"Internal":true,"Count Failed Values":true},{"ID":1022,"Name":"number of output rows","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1049,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1016,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1012,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Value":23,"Internal":true,"Count Failed Values":true},{"ID":1014,"Name":"time in aggregation build","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":963,"Name":"shuffle write time","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225868,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17503528,"Value":17503528,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4255703,"Value":4255703,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4255703,"Executor Run Time":50,"Executor CPU Time":17503528,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225874,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":991,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":987,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"23","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":992,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":955,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":960,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17516707,"Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":56,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3750230,"Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3750230,"Executor Run Time":56,"Executor CPU Time":17516707,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Completion Time":1596020225874,"Accumulables":[{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":992,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1001,"Name":"count of cache hit on states cache in provider","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":986,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1007,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":997,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":955,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":991,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1072,"Name":"internal.metrics.executorRunTime","Value":106,"Internal":true,"Count Failed Values":true},{"ID":1000,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1074,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":960,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":987,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":999,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1596020225875,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":20,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#130, count#131]\nArguments: [value#130, count#131]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#130, count#131]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1095,"metricType":"sum"}]},"time":1596020225891} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":20,"time":1596020225896} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":19,"time":1596020225897} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":18,"time":1596020225897} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.562Z","batchId":6,"batchDuration":351,"durationMs":{"triggerExecution":351,"queryPlanning":28,"getBatch":1,"latestOffset":6,"addBatch":273,"walCommit":25},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":24,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48837}}","endOffset":"{\"test5\":{\"0\":48881}}","numInputRows":44,"inputRowsPerSecond":100.22779043280183,"processedRowsPerSecond":125.35612535612536}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":21,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1759]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225988} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":22,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020226019} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1596020226076,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[15,14],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226116,"Failed":false,"Killed":false,"Accumulables":[{"ID":1120,"Name":"shuffle write time","Update":"543034","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1167,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1171,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1176,"Name":"time in aggregation build","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1178,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1179,"Name":"number of output rows","Update":"36","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Update":543034,"Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1206,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1205,"Name":"internal.metrics.executorCpuTime","Update":19652237,"Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1204,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2829254,"Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2829254,"Executor Run Time":19,"Executor CPU Time":19652237,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543034,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":36},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Completion Time":1596020226117,"Accumulables":[{"ID":1205,"Name":"internal.metrics.executorCpuTime","Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1178,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1204,"Name":"internal.metrics.executorRunTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1171,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1120,"Name":"shuffle write time","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1167,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1176,"Name":"time in aggregation build","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1206,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Value":36,"Internal":true,"Count Failed Values":true},{"ID":1179,"Name":"number of output rows","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226196,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1148,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1152,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1112,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1116,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1115,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1117,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":19415818,"Value":19415818,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":60,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3845429,"Value":3845429,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3845429,"Executor Run Time":60,"Executor CPU Time":19415818,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226204,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"48","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":14652861,"Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":65,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3933877,"Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3933877,"Executor Run Time":65,"Executor CPU Time":14652861,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Completion Time":1596020226204,"Accumulables":[{"ID":1115,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1231,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1153,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1117,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1147,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":1158,"Name":"count of cache hit on states cache in provider","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1152,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1116,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1143,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1148,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Value":125,"Internal":true,"Count Failed Values":true},{"ID":1112,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1596020226204,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":23,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#144, count#145]\nArguments: [value#144, count#145]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#144, count#145]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1252,"metricType":"sum"}]},"time":1596020226221} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":23,"time":1596020226230} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":22,"time":1596020226231} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":21,"time":1596020226231} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.916Z","batchId":7,"batchDuration":341,"durationMs":{"triggerExecution":341,"queryPlanning":24,"getBatch":0,"latestOffset":3,"addBatch":271,"walCommit":14},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":28,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48881}}","endOffset":"{\"test5\":{\"0\":48917}}","numInputRows":36,"inputRowsPerSecond":101.69491525423729,"processedRowsPerSecond":105.57184750733137}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1596020226301} diff --git a/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala b/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala new file mode 100644 index 000000000000..f73305b1b001 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala @@ -0,0 +1,40 @@ +/* + * 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.deploy.history + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.History.HISTORY_LOG_DIR +import org.apache.spark.util.ManualClock + +object Utils { + def withFsHistoryProvider(logDir: String)(fn: FsHistoryProvider => Unit): Unit = { + var provider: FsHistoryProvider = null + try { + val clock = new ManualClock() + val conf = new SparkConf().set(HISTORY_LOG_DIR, logDir) + val provider = new FsHistoryProvider(conf, clock) + provider.checkForLogs() + fn(provider) + } finally { + if (provider != null) { + provider.stop() + provider = null + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala new file mode 100644 index 000000000000..160535ea4d04 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala @@ -0,0 +1,63 @@ +/* + * 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.sql.streaming.ui + +import java.util.Locale +import javax.servlet.http.HttpServletRequest + +import org.mockito.Mockito.{mock, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.deploy.history.{Utils => HsUtils} +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore +import org.apache.spark.sql.test.SharedSparkSession + +class StreamingQueryHistorySuite extends SharedSparkSession with BeforeAndAfter { + + test("support streaming query events") { + val logDir = Thread.currentThread().getContextClassLoader.getResource("spark-events").toString + HsUtils.withFsHistoryProvider(logDir) { provider => + val appUi = provider.getAppUI("local-1596020211915", None).getOrElse { + assert(false, "Failed to load event log of local-1596020211915.") + null + } + assert(appUi.ui.appName == "StructuredKafkaWordCount") + assert(appUi.ui.store.store.count(classOf[StreamingQueryData]) == 1) + assert(appUi.ui.store.store.count(classOf[StreamingQueryProgressWrapper]) == 8) + + val store = new StreamingQueryStatusStore(appUi.ui.store.store) + val tab = new StreamingQueryTab(store, appUi.ui) + val request = mock(classOf[HttpServletRequest]) + var html = new StreamingQueryPage(tab).render(request) + .toString().toLowerCase(Locale.ROOT) + // 81.39: Avg Input /sec + assert(html.contains("81.39")) + // 157.05: Avg Process /sec + assert(html.contains("157.05")) + + val id = "8d268dc2-bc9c-4be8-97a9-b135d2943028" + val runId = "e225d92f-2545-48f8-87a2-9c0309580f8a" + when(request.getParameter("id")).thenReturn(runId) + html = new StreamingQueryStatisticsPage(tab).render(request) + .toString().toLowerCase(Locale.ROOT) + assert(html.contains("8 completed batches")) + assert(html.contains(id)) + assert(html.contains(runId)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala index c2b6688faf0e..246fa1f7c918 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.streaming.ui import java.util.{Locale, UUID} import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter -import scala.xml.Node import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.sql.streaming.StreamingQueryProgress import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.ui.SparkUI @@ -35,26 +37,26 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.statusListener).thenReturn(statusListener) + when(tab.store).thenReturn(store) val streamQuery = createStreamQueryUIData(id) - when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + when(store.allQueryUIData).thenReturn(Seq(streamQuery)) var html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("active streaming queries (1)")) - when(streamQuery.isActive).thenReturn(false) - when(streamQuery.exception).thenReturn(None) + when(streamQuery.summary.isActive).thenReturn(false) + when(streamQuery.summary.exception).thenReturn(None) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) assert(html.contains("finished")) - when(streamQuery.isActive).thenReturn(false) - when(streamQuery.exception).thenReturn(Option("exception in query")) + when(streamQuery.summary.isActive).thenReturn(false) + when(streamQuery.summary.exception).thenReturn(Option("exception in query")) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) @@ -66,17 +68,20 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) + when(request.getParameter("id")).thenReturn(id.toString) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(tab.store).thenReturn(store) val ui = mock(classOf[SparkUI]) when(request.getParameter("id")).thenReturn(id.toString) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.statusListener).thenReturn(statusListener) when(ui.conf).thenReturn(new SparkConf()) when(tab.parent).thenReturn(ui) val streamQuery = createStreamQueryUIData(id) - when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + when(store.allQueryUIData).thenReturn(Seq(streamQuery)) val html = renderStreamingQueryStatisticsPage(request, tab) .toString().toLowerCase(Locale.ROOT) @@ -94,15 +99,18 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { when(progress.batchId).thenReturn(2) when(progress.prettyJson).thenReturn("""{"a":1}""") + val summary = mock(classOf[StreamingQueryData], RETURNS_SMART_NULLS) + when(summary.isActive).thenReturn(true) + when(summary.name).thenReturn("query") + when(summary.id).thenReturn(id) + when(summary.runId).thenReturn(id) + when(summary.startTimestamp).thenReturn(1L) + when(summary.exception).thenReturn(None) + val streamQuery = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) - when(streamQuery.isActive).thenReturn(true) - when(streamQuery.name).thenReturn("query") - when(streamQuery.id).thenReturn(id) - when(streamQuery.runId).thenReturn(id) - when(streamQuery.startTimestamp).thenReturn(1L) + when(streamQuery.summary).thenReturn(summary) when(streamQuery.lastProgress).thenReturn(progress) when(streamQuery.recentProgress).thenReturn(Array(progress)) - when(streamQuery.exception).thenReturn(None) streamQuery } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala index 6aa440e5609c..91c55d5598a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala @@ -17,19 +17,28 @@ package org.apache.spark.sql.streaming.ui -import java.util.UUID +import java.text.SimpleDateFormat +import java.util.{Date, UUID} import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore +import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress, StreamTest} import org.apache.spark.sql.streaming +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore class StreamingQueryStatusListenerSuite extends StreamTest { test("onQueryStarted, onQueryProgress, onQueryTerminated") { - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) - // hanlde query started event + // handle query started event val id = UUID.randomUUID() val runId = UUID.randomUUID() val startEvent = new StreamingQueryListener.QueryStartedEvent( @@ -37,8 +46,9 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent) // result checking - assert(listener.activeQueryStatus.size() == 1) - assert(listener.activeQueryStatus.get(runId).name == "test") + assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => + uiData.summary.runId == runId && uiData.summary.name.equals("test"))) // handle query progress event val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) @@ -53,28 +63,32 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryProgress(processEvent) // result checking - val activeQuery = listener.activeQueryStatus.get(runId) - assert(activeQuery.isActive) - assert(activeQuery.recentProgress.length == 1) - assert(activeQuery.lastProgress.id == id) - assert(activeQuery.lastProgress.runId == runId) - assert(activeQuery.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") - assert(activeQuery.lastProgress.inputRowsPerSecond == 10.0) - assert(activeQuery.lastProgress.processedRowsPerSecond == 12.0) - assert(activeQuery.lastProgress.batchId == 2) - assert(activeQuery.lastProgress.prettyJson == """{"a":1}""") + val activeQuery = + queryStore.allQueryUIData.filter(_.summary.isActive).find(_.summary.runId == runId) + assert(activeQuery.isDefined) + assert(activeQuery.get.summary.isActive) + assert(activeQuery.get.recentProgress.length == 1) + assert(activeQuery.get.lastProgress.id == id) + assert(activeQuery.get.lastProgress.runId == runId) + assert(activeQuery.get.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") + assert(activeQuery.get.lastProgress.inputRowsPerSecond == 10.0) + assert(activeQuery.get.lastProgress.processedRowsPerSecond == 12.0) + assert(activeQuery.get.lastProgress.batchId == 2) + assert(activeQuery.get.lastProgress.prettyJson == """{"a":1}""") // handle terminate event val terminateEvent = new StreamingQueryListener.QueryTerminatedEvent(id, runId, None) listener.onQueryTerminated(terminateEvent) - assert(!listener.inactiveQueryStatus.head.isActive) - assert(listener.inactiveQueryStatus.head.runId == runId) - assert(listener.inactiveQueryStatus.head.id == id) + assert(!queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.isActive) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) } test("same query start multiple times") { - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) // handle first time start val id = UUID.randomUUID() @@ -94,11 +108,106 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent1) // result checking - assert(listener.activeQueryStatus.size() == 1) - assert(listener.inactiveQueryStatus.length == 1) - assert(listener.activeQueryStatus.containsKey(runId1)) - assert(listener.activeQueryStatus.get(runId1).id == id) - assert(listener.inactiveQueryStatus.head.runId == runId0) - assert(listener.inactiveQueryStatus.head.id == id) + assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).length == 1) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(_.summary.runId == runId1)) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => + uiData.summary.runId == runId1 && uiData.summary.id == id)) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId0) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) + } + + test("test small retained queries") { + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val conf = spark.sparkContext.conf + conf.set(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES.key, "2") + val listener = new StreamingQueryStatusListener(conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) + + def addNewQuery(): (UUID, UUID) = { + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + format.setTimeZone(getTimeZone("UTC")) + val id = UUID.randomUUID() + val runId = UUID.randomUUID() + val startEvent = new StreamingQueryListener.QueryStartedEvent( + id, runId, "test1", format.format(new Date(System.currentTimeMillis()))) + listener.onQueryStarted(startEvent) + (id, runId) + } + + def checkInactiveQueryStatus(numInactives: Int, targetInactives: Seq[UUID]): Unit = { + eventually(timeout(10.seconds)) { + val inactiveQueries = queryStore.allQueryUIData.filter(!_.summary.isActive) + assert(inactiveQueries.size == numInactives) + assert(inactiveQueries.map(_.summary.id).toSet == targetInactives.toSet) + } + } + + val (id1, runId1) = addNewQuery() + val (id2, runId2) = addNewQuery() + val (id3, runId3) = addNewQuery() + assert(queryStore.allQueryUIData.count(!_.summary.isActive) == 0) + + val terminateEvent1 = new StreamingQueryListener.QueryTerminatedEvent(id1, runId1, None) + listener.onQueryTerminated(terminateEvent1) + checkInactiveQueryStatus(1, Seq(id1)) + val terminateEvent2 = new StreamingQueryListener.QueryTerminatedEvent(id2, runId2, None) + listener.onQueryTerminated(terminateEvent2) + checkInactiveQueryStatus(2, Seq(id1, id2)) + val terminateEvent3 = new StreamingQueryListener.QueryTerminatedEvent(id3, runId3, None) + listener.onQueryTerminated(terminateEvent3) + checkInactiveQueryStatus(2, Seq(id2, id3)) + } + + test("test small retained progress") { + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val conf = spark.sparkContext.conf + conf.set(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES.key, "5") + val listener = new StreamingQueryStatusListener(conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) + + val id = UUID.randomUUID() + val runId = UUID.randomUUID() + val startEvent = new StreamingQueryListener.QueryStartedEvent( + id, runId, "test", "2016-12-05T20:54:20.827Z") + listener.onQueryStarted(startEvent) + + var batchId: Int = 0 + + def addQueryProgress(): Unit = { + val progress = mockProgressData(id, runId) + val processEvent = new streaming.StreamingQueryListener.QueryProgressEvent(progress) + listener.onQueryProgress(processEvent) + } + + def mockProgressData(id: UUID, runId: UUID): StreamingQueryProgress = { + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + format.setTimeZone(getTimeZone("UTC")) + + val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) + when(progress.id).thenReturn(id) + when(progress.runId).thenReturn(runId) + when(progress.timestamp).thenReturn(format.format(new Date(System.currentTimeMillis()))) + when(progress.inputRowsPerSecond).thenReturn(10.0) + when(progress.processedRowsPerSecond).thenReturn(12.0) + when(progress.batchId).thenReturn(batchId) + when(progress.prettyJson).thenReturn("""{"a":1}""") + + batchId += 1 + progress + } + + def checkQueryProcessData(targetNum: Int): Unit = { + eventually(timeout(10.seconds)) { + assert(queryStore.getQueryProgressData(runId).size == targetNum) + } + } + + Array.tabulate(4) { _ => addQueryProgress() } + checkQueryProcessData(4) + addQueryProgress() + checkQueryProcessData(5) + addQueryProgress() + checkQueryProcessData(5) } } From 03c3611835df0bc0b1396efb3304e0c74d675ef3 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 7 Dec 2020 15:48:12 +0900 Subject: [PATCH 3/3] Try Serial test --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 72b2caf90715..d0d8f5f2adce 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -134,7 +134,7 @@ jobs: - name: Run tests run: | # Hive and SQL tests become flaky when running in parallel as it's too intensive. - if [[ "$MODULES_TO_TEST" == "hive" ]] || [[ "$MODULES_TO_TEST" == "sql" ]]; then export SERIAL_SBT_TESTS=1; fi + if [[ "$MODULES_TO_TEST" == "hive" ]] || [[ "$MODULES_TO_TEST" == "sql" ]] || [[ "$MODULES_TO_TEST" == "hive-thriftserver" ]]; then export SERIAL_SBT_TESTS=1; fi ./dev/run-tests --parallelism 2 --modules "$MODULES_TO_TEST" --included-tags "$INCLUDED_TAGS" --excluded-tags "$EXCLUDED_TAGS" - name: Upload test results to report if: always()