-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-11206] Support SQL UI on the history server #9297
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
fdf9d28
ff4075d
b9870e6
3833055
c0abfc6
a5b1cf4
7b30bc7
d52288b
7a2aced
caab0ba
0af5afe
8d565f2
1954d71
927bae8
b03d98b
51f913b
bca3f5f
60033f8
fe5c165
8d94707
56f24ba
690277e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1 @@ | ||
| org.apache.spark.sql.execution.ui.SQLHistoryListenerFactory |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,6 @@ import java.beans.{BeanInfo, Introspector} | |
| import java.util.Properties | ||
| import java.util.concurrent.atomic.AtomicReference | ||
|
|
||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.immutable | ||
| import scala.reflect.runtime.universe.TypeTag | ||
|
|
@@ -1245,6 +1244,8 @@ object SQLContext { | |
| */ | ||
| @transient private val instantiatedContext = new AtomicReference[SQLContext]() | ||
|
|
||
| @transient private val sqlListener = new AtomicReference[SQLListener]() | ||
|
|
||
| /** | ||
| * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. | ||
| * | ||
|
|
@@ -1284,6 +1285,10 @@ object SQLContext { | |
| Option(instantiatedContext.get()) | ||
| } | ||
|
|
||
| private[sql] def clearSqlListener(): Unit = { | ||
| sqlListener.set(null) | ||
| } | ||
|
|
||
| /** | ||
| * Changes the SQLContext that will be returned in this thread and its children when | ||
| * SQLContext.getOrCreate() is called. This can be used to ensure that a given thread receives | ||
|
|
@@ -1332,9 +1337,13 @@ object SQLContext { | |
| * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. | ||
| */ | ||
| private[sql] def createListenerAndUI(sc: SparkContext): SQLListener = { | ||
| val listener = new SQLListener(sc.conf) | ||
| sc.addSparkListener(listener) | ||
| sc.ui.foreach(new SQLTab(listener, _)) | ||
| listener | ||
| if (sqlListener.get() == null) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Given that it's still legal to have multiple active SQLContexts in an application (although this is not a good practice), isn't this a user-facing behavioral change? If a user creates multiple SQLContexts, then they will now only have one SQL tab in the UI and one SQLListener, which is different than the old behavior.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also, is the only purpose of |
||
| val listener = new SQLListener(sc.conf) | ||
| if (sqlListener.compareAndSet(null, listener)) { | ||
| sc.addSparkListener(listener) | ||
| sc.ui.foreach(new SQLTab(listener, _)) | ||
| } | ||
| } | ||
| sqlListener.get() | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,8 @@ import java.util.concurrent.atomic.AtomicLong | |
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.sql.SQLContext | ||
| import org.apache.spark.sql.execution.ui.SparkPlanGraph | ||
| import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionStart, | ||
| SparkListenerSQLExecutionEnd} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| private[sql] object SQLExecution { | ||
|
|
@@ -45,25 +46,14 @@ private[sql] object SQLExecution { | |
| sc.setLocalProperty(EXECUTION_ID_KEY, executionId.toString) | ||
| val r = try { | ||
| val callSite = Utils.getCallSite() | ||
| sqlContext.listener.onExecutionStart( | ||
| executionId, | ||
| callSite.shortForm, | ||
| callSite.longForm, | ||
| queryExecution.toString, | ||
| SparkPlanGraph(queryExecution.executedPlan), | ||
| System.currentTimeMillis()) | ||
| sqlContext.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll summon @marmbrus to see whether the change in semantics here is ok. The events would now be processed asynchronously instead of in the same thread.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. /cc @zsxwing @andrewor14
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks good to me, as |
||
| executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, | ||
| SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) | ||
| try { | ||
| body | ||
| } finally { | ||
| // Ideally, we need to make sure onExecutionEnd happens after onJobStart and onJobEnd. | ||
| // However, onJobStart and onJobEnd run in the listener thread. Because we cannot add new | ||
| // SQL event types to SparkListener since it's a public API, we cannot guarantee that. | ||
| // | ||
| // SQLListener should handle the case that onExecutionEnd happens before onJobEnd. | ||
| // | ||
| // The worst case is onExecutionEnd may happen before onJobStart when the listener thread | ||
| // is very busy. If so, we cannot track the jobs for the execution. It seems acceptable. | ||
| sqlContext.listener.onExecutionEnd(executionId, System.currentTimeMillis()) | ||
| sqlContext.sparkContext.listenerBus.post(SparkListenerSQLExecutionEnd( | ||
| executionId, System.currentTimeMillis())) | ||
| } | ||
| } finally { | ||
| sc.setLocalProperty(EXECUTION_ID_KEY, null) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,46 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.sql.execution.metric.SQLMetricInfo | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Stores information about a SQL SparkPlan. | ||
| */ | ||
| @DeveloperApi | ||
| class SparkPlanInfo( | ||
| val nodeName: String, | ||
| val simpleString: String, | ||
| val children: Seq[SparkPlanInfo], | ||
| val metrics: Seq[SQLMetricInfo]) | ||
|
|
||
| private[sql] object SparkPlanInfo { | ||
|
|
||
| def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { | ||
| val metrics = plan.metrics.toSeq.map { case (key, metric) => | ||
| new SQLMetricInfo(metric.name.getOrElse(key), metric.id, | ||
| Utils.getFormattedClassName(metric.param)) | ||
| } | ||
| val children = plan.children.map(fromSparkPlan) | ||
|
|
||
| new SparkPlanInfo(plan.nodeName, plan.simpleString, children, metrics) | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,30 @@ | ||
| /* | ||
| * 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.metric | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Stores information about a SQL Metric. | ||
| */ | ||
| @DeveloperApi | ||
| class SQLMetricInfo( | ||
| val name: String, | ||
| val accumulatorId: Long, | ||
| val metricParam: String) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this method is going to be a no-op under certain circumstances then those circumstances should be documented here.