Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
7570eab
[SPARK-22788][STREAMING] Use correct hadoop config for fs append supp…
Dec 20, 2017
7798c9e
[SPARK-22824] Restore old offset for binary compatibility
jose-torres Dec 20, 2017
d762d11
[SPARK-22832][ML] BisectingKMeans unpersist unused datasets
zhengruifeng Dec 20, 2017
c89b431
[SPARK-22849] ivy.retrieve pattern should also consider `classifier`
gatorsmile Dec 20, 2017
792915c
[SPARK-22830] Scala Coding style has been improved in Spark Examples
chetkhatri Dec 20, 2017
b176014
[SPARK-22847][CORE] Remove redundant code in AppStatusListener while …
Ngone51 Dec 20, 2017
0114c89
[SPARK-22845][SCHEDULER] Modify spark.kubernetes.allocation.batch.del…
foxish Dec 21, 2017
fb0562f
[SPARK-22810][ML][PYSPARK] Expose Python API for LinearRegression wit…
yanboliang Dec 21, 2017
9c289a5
[SPARK-22387][SQL] Propagate session configs to data source read/writ…
jiangxb1987 Dec 21, 2017
d3ae3e1
[SPARK-19634][SQL][ML][FOLLOW-UP] Improve interface of dataframe vect…
WeichenXu123 Dec 21, 2017
cb9fc8d
[SPARK-22848][SQL] Eliminate mutable state from Stack
kiszk Dec 21, 2017
59d5263
[SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0
BryanCutler Dec 21, 2017
0abaf31
[SPARK-22852][BUILD] Exclude -Xlint:unchecked from sbt javadoc flags
easel Dec 21, 2017
4c2efde
[SPARK-22855][BUILD] Add -no-java-comments to sbt docs/scalacOptions
easel Dec 21, 2017
8a0ed5a
[SPARK-22668][SQL] Ensure no global variables in arguments of method …
cloud-fan Dec 21, 2017
d3a1d95
[SPARK-22786][SQL] only use AppStatusPlugin in history server
cloud-fan Dec 21, 2017
4e107fd
[SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPr…
wangyum Dec 21, 2017
fe65361
[SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when ch…
tejasapatil Dec 21, 2017
7beb375
[SPARK-22861][SQL] SQLAppStatusListener handles multi-job executions.
squito Dec 21, 2017
7ab165b
[SPARK-22648][K8S] Spark on Kubernetes - Documentation
foxish Dec 22, 2017
c0abb1d
[SPARK-22854][UI] Read Spark version from event logs.
Dec 22, 2017
c6f01ca
[SPARK-22750][SQL] Reuse mutable states when possible
mgaido91 Dec 22, 2017
a36b78b
[SPARK-22450][CORE][MLLIB][FOLLOWUP] safely register class for mllib …
zhengruifeng Dec 22, 2017
22e1849
[SPARK-22866][K8S] Fix path issue in Kubernetes dockerfile
foxish Dec 22, 2017
8df1da3
[SPARK-22862] Docs on lazy elimination of columns missing from an enc…
marmbrus Dec 22, 2017
13190a4
[SPARK-22874][PYSPARK][SQL] Modify checking pandas version to use Loo…
ueshin Dec 22, 2017
d23dc5b
[SPARK-22346][ML] VectorSizeHint Transformer for using VectorAssemble…
MrBago Dec 22, 2017
d3cbbdd
[SPARK-22757][Kubernetes] Enable use of remote dependencies in Kubern…
liyinan926 Dec 12, 2017
5d2cbc8
Addressed first round of comments
liyinan926 Dec 15, 2017
4ee76af
Addressed the second round of comments
liyinan926 Dec 16, 2017
9c8051a
Create one task per jar/file to download in the init-container
liyinan926 Dec 16, 2017
1f65417
More review comments
liyinan926 Dec 18, 2017
109ad80
Shorten variable names
liyinan926 Dec 19, 2017
c21fdcf
Removed traits that have only a single implementation
liyinan926 Dec 19, 2017
a3cd71d
Remove unused class arguments
liyinan926 Dec 19, 2017
23c5cd9
Improved documentation
liyinan926 Dec 19, 2017
2ec15c4
Addressed latest round of comments
liyinan926 Dec 20, 2017
5d1f889
Addressed more comments
liyinan926 Dec 21, 2017
9d9c841
Updated names of two configuration properties
liyinan926 Dec 22, 2017
c51bc56
Addressed more comments
liyinan926 Dec 25, 2017
28343fb
Addressed one more comment
liyinan926 Dec 26, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-22786][SQL] only use AppStatusPlugin in history server
## What changes were proposed in this pull request?

In apache#19681 we introduced a new interface called `AppStatusPlugin`, to register listeners and set up the UI for both live and history UI.

However I think it's an overkill for live UI. For example, we should not register `SQLListener` if users are not using SQL functions. Previously we register the `SQLListener` and set up SQL tab when `SparkSession` is firstly created, which indicates users are going to use SQL functions. But in apache#19681 , we register the SQL functions during `SparkContext` creation. The same thing should apply to streaming too.

I think we should keep the previous behavior, and only use this new interface for history server.

To reflect this change, I also rename the new interface to `SparkHistoryUIPlugin`

This PR also refines the tests for sql listener.

## How was this patch tested?

existing tests

Author: Wenchen Fan <[email protected]>

Closes apache#19981 from cloud-fan/listener.
  • Loading branch information
cloud-fan committed Dec 21, 2017
commit d3a1d9527bcd6675cc45773f01d4558cf4b46b3d
16 changes: 6 additions & 10 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend}
import org.apache.spark.scheduler.local.LocalSchedulerBackend
import org.apache.spark.status.{AppStatusPlugin, AppStatusStore}
import org.apache.spark.status.AppStatusStore
import org.apache.spark.storage._
import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump
import org.apache.spark.ui.{ConsoleProgressBar, SparkUI}
Expand Down Expand Up @@ -416,7 +416,8 @@ class SparkContext(config: SparkConf) extends Logging {

// Initialize the app status store and listener before SparkEnv is created so that it gets
// all events.
_statusStore = AppStatusStore.createLiveStore(conf, l => listenerBus.addToStatusQueue(l))
_statusStore = AppStatusStore.createLiveStore(conf)
listenerBus.addToStatusQueue(_statusStore.listener.get)

// Create the Spark execution environment (cache, map output tracker, etc)
_env = createSparkEnv(_conf, isLocal, listenerBus)
Expand Down Expand Up @@ -445,14 +446,9 @@ class SparkContext(config: SparkConf) extends Logging {
// For tests, do not enable the UI
None
}
_ui.foreach { ui =>
// Load any plugins that might want to modify the UI.
AppStatusPlugin.loadPlugins().foreach(_.setupUI(ui))

// Bind the UI before starting the task scheduler to communicate
// the bound port to the cluster manager properly
ui.bind()
}
// Bind the UI before starting the task scheduler to communicate
// the bound port to the cluster manager properly
_ui.foreach(_.bind())

_hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ import org.apache.spark.scheduler.ReplayListenerBus._
import org.apache.spark.status._
import org.apache.spark.status.KVUtils._
import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo}
import org.apache.spark.status.config._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
import org.apache.spark.util.kvstore._
Expand Down Expand Up @@ -322,15 +321,18 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
(new InMemoryStore(), true)
}

val plugins = ServiceLoader.load(
classOf[AppHistoryServerPlugin], Utils.getContextOrSparkClassLoader).asScala
val trackingStore = new ElementTrackingStore(kvstore, conf)
if (needReplay) {
val replayBus = new ReplayListenerBus()
val listener = new AppStatusListener(trackingStore, conf, false,
lastUpdateTime = Some(attempt.info.lastUpdated.getTime()))
replayBus.addListener(listener)
AppStatusPlugin.loadPlugins().foreach { plugin =>
plugin.setupListeners(conf, trackingStore, l => replayBus.addListener(l), false)
}
for {
plugin <- plugins
listener <- plugin.createListeners(conf, trackingStore)
} replayBus.addListener(listener)
try {
val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath))
replay(fileStatus, isApplicationCompleted(fileStatus), replayBus)
Expand All @@ -353,9 +355,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
HistoryServer.getAttemptURI(appId, attempt.info.attemptId),
attempt.info.startTime.getTime(),
attempt.info.appSparkVersion)
AppStatusPlugin.loadPlugins().foreach { plugin =>
plugin.setupUI(ui)
}
plugins.foreach(_.setupUI(ui))

val loadedUI = LoadedAppUI(ui)

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.status

import org.apache.spark.SparkConf
import org.apache.spark.scheduler.SparkListener
import org.apache.spark.ui.SparkUI

/**
* An interface for creating history listeners(to replay event logs) defined in other modules like
* SQL, and setup the UI of the plugin to rebuild the history UI.
*/
private[spark] trait AppHistoryServerPlugin {
/**
* Creates listeners to replay the event logs.
*/
def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener]

/**
* Sets up UI of this plugin to rebuild the history UI.
*/
def setupUI(ui: SparkUI): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ private[spark] class AppStatusListener(

import config._

private var sparkVersion = SPARK_VERSION
private val sparkVersion = SPARK_VERSION
private var appInfo: v1.ApplicationInfo = null
private var appSummary = new AppSummary(0, 0)
private var coresPerTask: Int = 1
Expand Down
71 changes: 0 additions & 71 deletions core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala

This file was deleted.

17 changes: 4 additions & 13 deletions core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,24 +17,22 @@

package org.apache.spark.status

import java.io.File
import java.util.{Arrays, List => JList}
import java.util.{List => JList}

import scala.collection.JavaConverters._

import org.apache.spark.{JobExecutionStatus, SparkConf}
import org.apache.spark.scheduler.SparkListener
import org.apache.spark.status.api.v1
import org.apache.spark.ui.scope._
import org.apache.spark.util.{Distribution, Utils}
import org.apache.spark.util.Distribution
import org.apache.spark.util.kvstore.{InMemoryStore, KVStore}

/**
* A wrapper around a KVStore that provides methods for accessing the API data stored within.
*/
private[spark] class AppStatusStore(
val store: KVStore,
listener: Option[AppStatusListener] = None) {
val listener: Option[AppStatusListener] = None) {

def applicationInfo(): v1.ApplicationInfo = {
store.view(classOf[ApplicationInfoWrapper]).max(1).iterator().next().info
Expand Down Expand Up @@ -346,17 +344,10 @@ private[spark] object AppStatusStore {

/**
* Create an in-memory store for a live application.
*
* @param conf Configuration.
* @param addListenerFn Function to register a listener with a bus.
*/
def createLiveStore(conf: SparkConf, addListenerFn: SparkListener => Unit): AppStatusStore = {
def createLiveStore(conf: SparkConf): AppStatusStore = {
val store = new ElementTrackingStore(new InMemoryStore(), conf)
val listener = new AppStatusListener(store, conf, true)
addListenerFn(listener)
AppStatusPlugin.loadPlugins().foreach { p =>
p.setupListeners(conf, store, addListenerFn, true)
}
new AppStatusStore(store, listener = Some(listener))
}

Expand Down
20 changes: 9 additions & 11 deletions core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,13 @@ import javax.servlet.http.HttpServletRequest

import scala.xml.Node

import org.mockito.Matchers.anyString
import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS}

import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler._
import org.apache.spark.status.AppStatusStore
import org.apache.spark.ui.jobs.{StagePage, StagesTab}
import org.apache.spark.util.Utils

class StagePageSuite extends SparkFunSuite with LocalSparkContext {

Expand All @@ -55,20 +53,20 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext {
* This also runs a dummy stage to populate the page with useful content.
*/
private def renderStagePage(conf: SparkConf): Seq[Node] = {
val bus = new ReplayListenerBus()
val store = AppStatusStore.createLiveStore(conf, l => bus.addListener(l))
val statusStore = AppStatusStore.createLiveStore(conf)
val listener = statusStore.listener.get

try {
val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS)
when(tab.store).thenReturn(store)
when(tab.store).thenReturn(statusStore)

val request = mock(classOf[HttpServletRequest])
when(tab.conf).thenReturn(conf)
when(tab.appName).thenReturn("testing")
when(tab.headerTabs).thenReturn(Seq.empty)
when(request.getParameter("id")).thenReturn("0")
when(request.getParameter("attempt")).thenReturn("0")
val page = new StagePage(tab, store)
val page = new StagePage(tab, statusStore)

// Simulate a stage in job progress listener
val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details")
Expand All @@ -77,17 +75,17 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext {
taskId =>
val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY,
false)
bus.postToAll(SparkListenerStageSubmitted(stageInfo))
bus.postToAll(SparkListenerTaskStart(0, 0, taskInfo))
listener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo))
listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo))
taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis())
val taskMetrics = TaskMetrics.empty
taskMetrics.incPeakExecutionMemory(peakExecutionMemory)
bus.postToAll(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics))
listener.onTaskEnd(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics))
}
bus.postToAll(SparkListenerStageCompleted(stageInfo))
listener.onStageCompleted(SparkListenerStageCompleted(stageInfo))
page.render(request)
} finally {
store.close()
statusStore.close()
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,11 @@ import org.apache.spark.sql.execution.metric._
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity}
import org.apache.spark.status.config._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.kvstore.KVStore

private[sql] class SQLAppStatusListener(
class SQLAppStatusListener(
conf: SparkConf,
kvstore: ElementTrackingStore,
live: Boolean,
ui: Option[SparkUI] = None)
extends SparkListener with Logging {
live: Boolean) extends SparkListener with Logging {

// How often to flush intermediate state of a live execution to the store. When replaying logs,
// never flush (only do the very last write).
Expand All @@ -50,7 +46,10 @@ private[sql] class SQLAppStatusListener(
private val liveExecutions = new ConcurrentHashMap[Long, LiveExecutionData]()
private val stageMetrics = new ConcurrentHashMap[Int, LiveStageMetrics]()

private var uiInitialized = false
// Returns true if this listener has no live data. Exposed for tests only.
private[sql] def noLiveData(): Boolean = {
liveExecutions.isEmpty && stageMetrics.isEmpty
}

kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get(UI_RETAINED_EXECUTIONS)) { count =>
cleanupExecutions(count)
Expand Down Expand Up @@ -230,14 +229,6 @@ private[sql] class SQLAppStatusListener(
}

private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = {
// Install the SQL tab in a live app if it hasn't been initialized yet.
if (!uiInitialized) {
ui.foreach { _ui =>
new SQLTab(new SQLAppStatusStore(kvstore, Some(this)), _ui)
}
uiInitialized = true
}

val SparkListenerSQLExecutionStart(executionId, description, details,
physicalPlanDescription, sparkPlanInfo, time) = event

Expand Down Expand Up @@ -389,7 +380,7 @@ private class LiveStageMetrics(
val accumulatorIds: Array[Long],
val taskMetrics: ConcurrentHashMap[Long, LiveTaskMetrics])

private[sql] class LiveTaskMetrics(
private class LiveTaskMetrics(
val ids: Array[Long],
val values: Array[Long],
val succeeded: Boolean)
Loading