From fdf9d28362fe991a1df0d5392c9021db78fa7541 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 16 Oct 2015 15:00:11 +0800 Subject: [PATCH 01/22] Update SparkListener to handle other events --- .../spark/scheduler/SparkListener.scala | 7 +- .../spark/scheduler/SparkListenerBus.scala | 1 + .../spark/sql/execution/SQLExecution.scala | 23 ++---- .../spark/sql/execution/ui/SQLListener.scala | 76 +++++++++++-------- 4 files changed, 58 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 896f1743332f..7441bdad38ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.util.{Distribution, Utils} @DeveloperApi -sealed trait SparkListenerEvent +trait SparkListenerEvent @DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) @@ -223,6 +223,11 @@ trait SparkListener { * Called when the driver receives a block update info. */ def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated) { } + + /** + * Called when other events like SQL-specific events are posted. + */ + def onOtherEvent(event: SparkListenerEvent) {} } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 04afde33f5aa..027947c295e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,7 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) case logStart: SparkListenerLogStart => // ignore event log metadata + case _ => listener.onOtherEvent(_) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 1422e15549c9..a4454fa53f14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -21,7 +21,7 @@ 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.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart, SparkPlanGraph} import org.apache.spark.util.Utils private[sql] object SQLExecution { @@ -45,25 +45,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( + executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, + SparkPlanGraph(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) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 5a072de400b6..890ed9a04c6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -25,6 +25,18 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} +private[sql] case class SparkListenerSQLExecutionStart( + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + physicalPlanGraph: SparkPlanGraph, + time: Long) + extends SparkListenerEvent + +private[sql] case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) + extends SparkListenerEvent + private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) @@ -193,37 +205,39 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } } - def onExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - physicalPlanGraph: SparkPlanGraph, - time: Long): Unit = { - val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) - } - - val executionUIData = new SQLExecutionUIData(executionId, description, details, - physicalPlanDescription, physicalPlanGraph, sqlPlanMetrics.toMap, time) - synchronized { - activeExecutions(executionId) = executionUIData - _executionIdToData(executionId) = executionUIData - } - } - - def onExecutionEnd(executionId: Long, time: Long): Unit = synchronized { - _executionIdToData.get(executionId).foreach { executionUIData => - executionUIData.completionTime = Some(time) - if (!executionUIData.hasRunningJobs) { - // onExecutionEnd happens after all "onJobEnd"s - // So we should update the execution lists. - markExecutionFinished(executionId) - } else { - // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. - // Then we don't if the execution is successful, so let the last onJobEnd updates the - // execution lists. - } + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case executionStart: SparkListenerSQLExecutionStart => + val sqlPlanMetrics = executionStart.physicalPlanGraph.nodes.flatMap { node => + node.metrics.map(metric => metric.accumulatorId -> metric) + } + val executionUIData = new SQLExecutionUIData( + executionStart.executionId, + executionStart.description, + executionStart.details, + executionStart.physicalPlanDescription, + executionStart.physicalPlanGraph, + sqlPlanMetrics.toMap, + executionStart.time) + synchronized { + activeExecutions(executionStart.executionId) = executionUIData + _executionIdToData(executionStart.executionId) = executionUIData + } + case executionEnd: SparkListenerSQLExecutionEnd => + synchronized { + _executionIdToData.get(executionEnd.executionId).foreach { executionUIData => + executionUIData.completionTime = Some(executionEnd.time) + if (!executionUIData.hasRunningJobs) { + // onExecutionEnd happens after all "onJobEnd"s + // So we should update the execution lists. + markExecutionFinished(executionEnd.executionId) + } else { + // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. + // Then we don't if the execution is successful, so let the last onJobEnd updates the + // execution lists. + } + } + } } } From ff4075d89b480d99d0320390d00a5b10aadc9a93 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 20 Oct 2015 14:06:07 +0800 Subject: [PATCH 02/22] Write sql events to event log --- .../org/apache/spark/JavaSparkListener.java | 3 + .../apache/spark/SparkFirehoseListener.java | 4 ++ .../scheduler/EventLoggingListener.scala | 4 ++ .../spark/scheduler/SparkListener.scala | 20 +++++- .../spark/scheduler/SparkListenerBus.scala | 2 +- .../org/apache/spark/sql/SparkPlanInfo.scala | 33 +++++++++ .../org/apache/spark/sql/SqlMetricInfo.scala | 28 ++++++++ .../org/apache/spark/util/JsonProtocol.scala | 72 +++++++++++++++++++ .../spark/sql/execution/SQLExecution.scala | 4 +- .../spark/sql/execution/SparkPlanInfo.scala | 33 +++++++++ .../sql/execution/metric/SQLMetrics.scala | 16 ++++- .../spark/sql/execution/ui/SQLListener.scala | 17 +---- .../sql/execution/ui/SparkPlanGraph.scala | 22 +++--- .../execution/metric/SQLMetricsSuite.scala | 5 +- .../sql/execution/ui/SQLListenerSuite.scala | 41 ++++++----- 15 files changed, 256 insertions(+), 48 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala create mode 100644 core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java index fa9acf0a15b8..23bc9a2e8172 100644 --- a/core/src/main/java/org/apache/spark/JavaSparkListener.java +++ b/core/src/main/java/org/apache/spark/JavaSparkListener.java @@ -82,4 +82,7 @@ public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } @Override public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { } + @Override + public void onOtherEvent(SparkListenerEvent event) { } + } diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 1214d05ba606..e6b24afd88ad 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -118,4 +118,8 @@ public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { onEvent(blockUpdated); } + @Override + public void onOtherEvent(SparkListenerEvent event) { + onEvent(event); + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 000a021a528c..eaa07acc5132 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -207,6 +207,10 @@ private[spark] class EventLoggingListener( // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } + override def onOtherEvent(event: SparkListenerEvent): Unit = { + logEvent(event, flushLogger = true) + } + /** * Stop logging events. The event log file will be renamed so that it loses the * ".inprogress" suffix. diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 7441bdad38ad..ddd5b9fd11c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler import java.util.Properties +import org.apache.spark.sql.SparkPlanInfo + import scala.collection.Map import scala.collection.mutable @@ -30,7 +32,7 @@ import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.util.{Distribution, Utils} @DeveloperApi -trait SparkListenerEvent +sealed trait SparkListenerEvent @DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) @@ -124,6 +126,20 @@ case class SparkListenerApplicationStart( @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerSQLExecutionStart( + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo, + time: Long) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) + extends SparkListenerEvent + /** * An internal class that describes the metadata of an event log. * This event is not meant to be posted to listeners downstream. @@ -227,7 +243,7 @@ trait SparkListener { /** * Called when other events like SQL-specific events are posted. */ - def onOtherEvent(event: SparkListenerEvent) {} + def onOtherEvent(event: SparkListenerEvent) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 027947c295e6..95722a07144e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,7 +61,7 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) case logStart: SparkListenerLogStart => // ignore event log metadata - case _ => listener.onOtherEvent(_) + case _ => listener.onOtherEvent(event) } } diff --git a/core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala b/core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala new file mode 100644 index 000000000000..d84befeea812 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala @@ -0,0 +1,33 @@ +/* + * 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 + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Stores information about a SQL SparkPlan. + */ +@DeveloperApi +class SparkPlanInfo( + val nodeName: String, + val simpleString: String, + val children: Seq[SparkPlanInfo], + val metrics: Seq[SqlMetricInfo] +) { +} diff --git a/core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala b/core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala new file mode 100644 index 000000000000..46b35ff156a9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala @@ -0,0 +1,28 @@ +/* + * 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 + +/** + * :: DeveloperApi :: + * Stores information about a SQL Metric. + */ +class SqlMetricInfo( + val name: String, + val accumulatorId: Long, + val metricParam: String) { +} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a06dc6f709d3..a29c42593e6c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import java.util.{Properties, UUID} import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.sql.{SqlMetricInfo, SparkPlanInfo} import scala.collection.JavaConverters._ import scala.collection.Map @@ -94,6 +95,10 @@ private[spark] object JsonProtocol { logStartToJson(logStart) case metricsUpdate: SparkListenerExecutorMetricsUpdate => executorMetricsUpdateToJson(metricsUpdate) + case sqlExecutionStart: SparkListenerSQLExecutionStart => + sqlExecutionStartToJson(sqlExecutionStart) + case sqlExecutionEnd: SparkListenerSQLExecutionEnd => + sqlExecutionEndToJson(sqlExecutionEnd) case blockUpdated: SparkListenerBlockUpdated => throw new MatchError(blockUpdated) // TODO(ekl) implement this } @@ -239,6 +244,22 @@ private[spark] object JsonProtocol { }) } + def sqlExecutionStartToJson(sqlExecutionStart: SparkListenerSQLExecutionStart): JValue = { + ("Event" -> Utils.getFormattedClassName(sqlExecutionStart)) ~ + ("Execution ID" -> sqlExecutionStart.executionId) ~ + ("Description" -> sqlExecutionStart.description) ~ + ("Details" -> sqlExecutionStart.details) ~ + ("Physical Plan Description" -> sqlExecutionStart.physicalPlanDescription) ~ + ("SparkPlan Info" -> sparkPlanInfoToJson(sqlExecutionStart.sparkPlanInfo)) ~ + ("Timestamp" -> sqlExecutionStart.time) + } + + def sqlExecutionEndToJson(sqlExecutionEnd: SparkListenerSQLExecutionEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(sqlExecutionEnd)) ~ + ("Execution ID" -> sqlExecutionEnd.executionId) ~ + ("Timestamp" -> sqlExecutionEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | * -------------------------------------------------------------------- */ @@ -428,6 +449,21 @@ private[spark] object JsonProtocol { ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) } + def sparkPlanInfoToJson(planInfo: SparkPlanInfo): JValue = { + val children = JArray(planInfo.children.map(sparkPlanInfoToJson).toList) + val metrics = JArray(planInfo.metrics.map(sqlMetricInfoToJson).toList) + ("Node Name" -> planInfo.nodeName) ~ + ("Simple String" -> planInfo.simpleString) ~ + ("Children" -> children) ~ + ("Metrics" -> metrics) + } + + def sqlMetricInfoToJson(sqlMetricInfo: SqlMetricInfo): JValue = { + ("Name" -> sqlMetricInfo.name) ~ + ("Accumulator Id" -> sqlMetricInfo.accumulatorId) ~ + ("Metric Param" -> sqlMetricInfo.metricParam) + } + /** ------------------------------ * * Util JSON serialization methods | * ------------------------------- */ @@ -485,6 +521,8 @@ private[spark] object JsonProtocol { val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) val logStart = Utils.getFormattedClassName(SparkListenerLogStart) val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) + val sqlExecutionStart = Utils.getFormattedClassName(SparkListenerSQLExecutionStart) + val sqlExecutionEnd = Utils.getFormattedClassName(SparkListenerSQLExecutionEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -504,9 +542,28 @@ private[spark] object JsonProtocol { case `executorRemoved` => executorRemovedFromJson(json) case `logStart` => logStartFromJson(json) case `metricsUpdate` => executorMetricsUpdateFromJson(json) + case `sqlExecutionStart` => sqlExecutionStartFromJson(json) + case `sqlExecutionEnd` => sqlExecutionEndFromJson(json) } } + def sqlExecutionStartFromJson(json: JValue): SparkListenerSQLExecutionStart = { + val executionId = (json \ "Execution ID").extract[Long] + val description = (json \ "Description").extract[String] + val details = (json \ "Details").extract[String] + val physicalPlanDescription = (json \ "Physical Plan Description").extract[String] + val sparkPlanInfo = sparkPlanInfoFromJson(json \ "SparkPlan Info") + val time = (json \ "Timestamp").extract[Long] + SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, + sparkPlanInfo, time) + } + + def sqlExecutionEndFromJson(json: JValue): SparkListenerSQLExecutionEnd = { + val executionId = (json \ "Execution ID").extract[Long] + val time = (json \ "Timestamp").extract[Long] + SparkListenerSQLExecutionEnd(executionId, time) + } + def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { val stageInfo = stageInfoFromJson(json \ "Stage Info") val properties = propertiesFromJson(json \ "Properties") @@ -895,6 +952,21 @@ private[spark] object JsonProtocol { new ExecutorInfo(executorHost, totalCores, logUrls) } + def sparkPlanInfoFromJson(json: JValue): SparkPlanInfo = { + val nodeName = (json \ "Node Name").extract[String] + val simpleString = (json \ "Simple String").extract[String] + val children = (json \ "Children").extract[List[JValue]].map(sparkPlanInfoFromJson) + val metrics = (json \ "Metrics").extract[List[JValue]].map(sqlMetricInfoFromJson) + new SparkPlanInfo(nodeName, simpleString, children, metrics) + } + + def sqlMetricInfoFromJson(json: JValue): SqlMetricInfo = { + val name = (json \ "Name").extract[String] + val accumulatorId = (json \ "Accumulator Id").extract[Long] + val metricParam = (json \ "Metric Param").extract[String] + new SqlMetricInfo(name, accumulatorId, metricParam) + } + /** -------------------------------- * * Util JSON deserialization methods | * --------------------------------- */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index a4454fa53f14..8152ec744144 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution import java.util.concurrent.atomic.AtomicLong import org.apache.spark.SparkContext +import org.apache.spark.scheduler.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart, SparkPlanGraph} import org.apache.spark.util.Utils private[sql] object SQLExecution { @@ -47,7 +47,7 @@ private[sql] object SQLExecution { val callSite = Utils.getCallSite() sqlContext.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, - SparkPlanGraph(queryExecution.executedPlan), System.currentTimeMillis())) + SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) try { body } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala new file mode 100644 index 000000000000..dd66e58284e0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -0,0 +1,33 @@ +/* + * 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.sql.{SqlMetricInfo, SparkPlanInfo} + +object SparkPlanInfo { + + def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { + //Todo correct the param + val metrics = plan.metrics.toSeq.map { case (key, metric) => + new SqlMetricInfo(metric.name.getOrElse(key), metric.id, metric.param.toString) + } + val children = plan.children.map(fromSparkPlan) + + new SparkPlanInfo(plan.nodeName, plan.simpleString, children, metrics) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 1c253e3942e9..6d961cff80b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -91,7 +91,21 @@ private[sql] class LongSQLMetric private[metric](name: String, param: LongSQLMet } } -private class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) +private[sql] object LongSQLMetricParam extends SQLMetricParam[LongSQLMetricValue, Long] { + + val stringValue : Seq[Long] => String = _.sum.toString + + override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) + + override def addInPlace(r1: LongSQLMetricValue, r2: LongSQLMetricValue): LongSQLMetricValue = + r1.add(r2.value) + + override def zero(initialValue: LongSQLMetricValue): LongSQLMetricValue = zero + + override def zero: LongSQLMetricValue = new LongSQLMetricValue(0L) +} + +private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) extends SQLMetricParam[LongSQLMetricValue, Long] { override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 890ed9a04c6b..8a9a4ff35e22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -25,18 +25,6 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} -private[sql] case class SparkListenerSQLExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - physicalPlanGraph: SparkPlanGraph, - time: Long) - extends SparkListenerEvent - -private[sql] case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent - private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) @@ -208,7 +196,8 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { case executionStart: SparkListenerSQLExecutionStart => - val sqlPlanMetrics = executionStart.physicalPlanGraph.nodes.flatMap { node => + val physicalPlanGraph = SparkPlanGraph(executionStart.sparkPlanInfo) + val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => node.metrics.map(metric => metric.accumulatorId -> metric) } val executionUIData = new SQLExecutionUIData( @@ -216,7 +205,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi executionStart.description, executionStart.details, executionStart.physicalPlanDescription, - executionStart.physicalPlanGraph, + physicalPlanGraph, sqlPlanMetrics.toMap, executionStart.time) synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index f1fce5478a3f..466681331166 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.ui import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.sql.SparkPlanInfo + import scala.collection.mutable import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} +import org.apache.spark.sql.execution.metric.{LongSQLMetricParam, SQLMetricParam, SQLMetricValue} /** * A graph used for storing information of an executionPlan of DataFrame. @@ -48,27 +50,29 @@ private[sql] object SparkPlanGraph { /** * Build a SparkPlanGraph from the root of a SparkPlan tree. */ - def apply(plan: SparkPlan): SparkPlanGraph = { + def apply(planInfo: SparkPlanInfo): SparkPlanGraph = { val nodeIdGenerator = new AtomicLong(0) val nodes = mutable.ArrayBuffer[SparkPlanGraphNode]() val edges = mutable.ArrayBuffer[SparkPlanGraphEdge]() - buildSparkPlanGraphNode(plan, nodeIdGenerator, nodes, edges) + buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges) new SparkPlanGraph(nodes, edges) } private def buildSparkPlanGraphNode( - plan: SparkPlan, + planInfo: SparkPlanInfo, nodeIdGenerator: AtomicLong, nodes: mutable.ArrayBuffer[SparkPlanGraphNode], edges: mutable.ArrayBuffer[SparkPlanGraphEdge]): SparkPlanGraphNode = { - val metrics = plan.metrics.toSeq.map { case (key, metric) => - SQLPlanMetric(metric.name.getOrElse(key), metric.id, - metric.param.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) + //TODO correct the metric Param + val metrics = planInfo.metrics.map { metric => + SQLPlanMetric(metric.name, metric.accumulatorId, + LongSQLMetricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) + // metric.metricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) } val node = SparkPlanGraphNode( - nodeIdGenerator.getAndIncrement(), plan.nodeName, plan.simpleString, metrics) + nodeIdGenerator.getAndIncrement(), planInfo.nodeName, planInfo.simpleString, metrics) nodes += node - val childrenNodes = plan.children.map( + val childrenNodes = planInfo.children.map( child => buildSparkPlanGraphNode(child, nodeIdGenerator, nodes, edges)) for (child <- childrenNodes) { edges += SparkPlanGraphEdge(child.id, node.id) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index cdd885ba1420..0d798554fa63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.metric import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import org.apache.spark.sql.execution.SparkPlanInfo + import scala.collection.mutable import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ @@ -84,7 +86,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { if (jobs.size == expectedNumOfJobs) { // If we can track all jobs, check the metric values val metricValues = sqlContext.listener.getExecutionMetrics(executionId) - val actualMetrics = SparkPlanGraph(df.queryExecution.executedPlan).nodes.filter { node => + val actualMetrics = SparkPlanGraph(SparkPlanInfo.fromSparkPlan( + df.queryExecution.executedPlan)).nodes.filter { node => expectedMetrics.contains(node.id) }.map { node => val nodeMetrics = node.metrics.map { metric => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index c15aac775096..b839db7eb6fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.sql.execution.metric.LongSQLMetricValue import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SQLContext} -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.test.SharedSQLContext class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { @@ -82,7 +82,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { val executionId = 0 val df = createTestDataFrame val accumulatorIds = - SparkPlanGraph(df.queryExecution.executedPlan).nodes.flatMap(_.metrics.map(_.accumulatorId)) + SparkPlanGraph(SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan)) + .nodes.flatMap(_.metrics.map(_.accumulatorId)) // Assume all accumulators are long var accumulatorValue = 0L val accumulatorUpdates = accumulatorIds.map { id => @@ -90,13 +91,13 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { (id, accumulatorValue) }.toMap - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) val executionUIData = listener.executionIdToData(0) @@ -206,7 +207,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { time = System.currentTimeMillis(), JobSucceeded )) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) assert(executionUIData.runningJobs.isEmpty) assert(executionUIData.succeededJobs === Seq(0)) @@ -219,19 +221,20 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { val listener = new SQLListener(sqlContext.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), @@ -248,13 +251,13 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { val listener = new SQLListener(sqlContext.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), @@ -271,7 +274,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 1, time = System.currentTimeMillis(), @@ -288,19 +292,20 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { val listener = new SQLListener(sqlContext.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Seq.empty, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), From b9870e6579a59628c82470e55f2cb6c4ec8fa2a7 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 21 Oct 2015 14:29:37 +0800 Subject: [PATCH 03/22] Move sql UI classes to core --- .../spark/scheduler/SparkListener.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 8 +++++++- .../spark/ui/sql}/AllExecutionsPage.scala | 9 ++++----- .../apache/spark/ui/sql}/ExecutionPage.scala | 10 ++++------ .../apache/spark/ui/sql}/SQLListener.scala | 14 ++++++------- .../org/apache/spark/ui/sql}/SQLMetrics.scala | 20 +++++++++---------- .../org/apache/spark/ui/sql}/SQLTab.scala | 13 +++++++----- .../apache/spark/ui/sql}/SparkPlanGraph.scala | 9 ++------- .../spark/{ => ui}/sql/SparkPlanInfo.scala | 2 +- .../spark/{ => ui}/sql/SqlMetricInfo.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/execution/Aggregate.scala | 3 +-- .../spark/sql/execution/SparkPlan.scala | 3 ++- .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../aggregate/SortBasedAggregate.scala | 2 +- .../SortBasedAggregationIterator.scala | 2 +- .../aggregate/TungstenAggregate.scala | 2 +- .../TungstenAggregationIterator.scala | 3 ++- .../spark/sql/execution/basicOperators.scala | 2 +- .../execution/joins/BroadcastHashJoin.scala | 3 ++- .../joins/BroadcastHashOuterJoin.scala | 3 ++- .../joins/BroadcastLeftSemiJoinHash.scala | 2 +- .../joins/BroadcastNestedLoopJoin.scala | 2 +- .../execution/joins/CartesianProduct.scala | 2 +- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../sql/execution/joins/HashOuterJoin.scala | 2 +- .../sql/execution/joins/HashSemiJoin.scala | 2 +- .../sql/execution/joins/HashedRelation.scala | 2 +- .../sql/execution/joins/LeftSemiJoinBNL.scala | 2 +- .../execution/joins/LeftSemiJoinHash.scala | 2 +- .../execution/joins/ShuffledHashJoin.scala | 2 +- .../joins/ShuffledHashOuterJoin.scala | 3 ++- .../sql/execution/joins/SortMergeJoin.scala | 3 ++- .../execution/joins/SortMergeOuterJoin.scala | 3 ++- .../org/apache/spark/sql/execution/sort.scala | 2 +- .../TungstenAggregationIteratorSuite.scala | 2 +- .../execution/joins/HashedRelationSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 6 ++++-- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 3 ++- 41 files changed, 87 insertions(+), 77 deletions(-) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/ui => core/src/main/scala/org/apache/spark/ui/sql}/AllExecutionsPage.scala (99%) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/ui => core/src/main/scala/org/apache/spark/ui/sql}/ExecutionPage.scala (95%) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/ui => core/src/main/scala/org/apache/spark/ui/sql}/SQLListener.scala (97%) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/metric => core/src/main/scala/org/apache/spark/ui/sql}/SQLMetrics.scala (87%) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/ui => core/src/main/scala/org/apache/spark/ui/sql}/SQLTab.scala (76%) rename {sql/core/src/main/scala/org/apache/spark/sql/execution/ui => core/src/main/scala/org/apache/spark/ui/sql}/SparkPlanGraph.scala (94%) rename core/src/main/scala/org/apache/spark/{ => ui}/sql/SparkPlanInfo.scala (97%) rename core/src/main/scala/org/apache/spark/{ => ui}/sql/SqlMetricInfo.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index ddd5b9fd11c9..b9e067af7924 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.sql.SparkPlanInfo +import org.apache.spark.ui.sql.SparkPlanInfo import scala.collection.Map import scala.collection.mutable diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 99085ada9f0a..3bed77ef77a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -30,6 +30,7 @@ import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.ui.scope.RDDOperationGraphListener +import org.apache.spark.ui.sql.{SQLTab, SQLListener} /** * Top level user interface for a Spark application. @@ -150,7 +151,12 @@ private[spark] object SparkUI { appName: String, basePath: String, startTime: Long): SparkUI = { - create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) + val sqlListener = new SQLListener(conf) + listenerBus.addListener(sqlListener) + val sparkUI = create( + None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) + new SQLTab(sqlListener, sparkUI) + sparkUI } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala rename to core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala index 49646a99d68c..64b4031a4972 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.ui +package org.apache.spark.ui.sql import javax.servlet.http.HttpServletRequest -import scala.collection.mutable -import scala.xml.Node - import org.apache.commons.lang3.StringEscapeUtils - import org.apache.spark.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} +import scala.collection.mutable +import scala.xml.Node + private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with Logging { private val listener = parent.listener diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala rename to core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala index e74d6fb396e1..0e3fbbebb5af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala @@ -15,18 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.ui +package org.apache.spark.ui.sql import javax.servlet.http.HttpServletRequest -import scala.xml.{Node, Unparsed} - -import org.apache.commons.lang3.StringEscapeUtils - import org.apache.spark.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} -private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { +import scala.xml.Node + +private[ui] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { private val listener = parent.listener diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala index 8a9a4ff35e22..73f2bbbbffdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.ui - -import scala.collection.mutable +package org.apache.spark.ui.sql import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} -private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { +import scala.collection.mutable + +private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Logging { + + val EXECUTION_ID_KEY = "spark.sql.execution.id" private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) @@ -78,7 +78,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } override def onJobStart(jobStart: SparkListenerJobStart): Unit = { - val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) + val executionIdString = jobStart.properties.getProperty(EXECUTION_ID_KEY) if (executionIdString == null) { // This is not a job created by SQL return diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala similarity index 87% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala index 6d961cff80b8..24c8288cd91d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.metric +package org.apache.spark.ui.sql +import org.apache.spark.{SparkContext, AccumulableParam, Accumulable} import org.apache.spark.util.Utils -import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} /** * Create a layer for specialized metric. We cannot add `@specialized` to @@ -26,7 +26,7 @@ import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} * * An implementation of SQLMetric should override `+=` and `add` to avoid boxing. */ -private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( +private[spark] abstract class SQLMetric[R <: SQLMetricValue[T], T]( name: String, val param: SQLMetricParam[R, T]) extends Accumulable[R, T](param.zero, param, Some(name), true) { @@ -39,7 +39,7 @@ private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. */ -private[sql] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends AccumulableParam[R, T] { +private[spark] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends AccumulableParam[R, T] { /** * A function that defines how we aggregate the final accumulator results among all tasks, @@ -54,7 +54,7 @@ private[sql] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends Accumulable * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. */ -private[sql] trait SQLMetricValue[T] extends Serializable { +private[spark] trait SQLMetricValue[T] extends Serializable { def value: T @@ -64,7 +64,7 @@ private[sql] trait SQLMetricValue[T] extends Serializable { /** * A wrapper of Long to avoid boxing and unboxing when using Accumulator */ -private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { +private[spark] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { def add(incr: Long): LongSQLMetricValue = { _value += incr @@ -79,7 +79,7 @@ private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetr * A specialized long Accumulable to avoid boxing and unboxing when using Accumulator's * `+=` and `add`. */ -private[sql] class LongSQLMetric private[metric](name: String, param: LongSQLMetricParam) +private[spark] class LongSQLMetric(name: String, param: LongSQLMetricParam) extends SQLMetric[LongSQLMetricValue, Long](name, param) { override def +=(term: Long): Unit = { @@ -91,7 +91,7 @@ private[sql] class LongSQLMetric private[metric](name: String, param: LongSQLMet } } -private[sql] object LongSQLMetricParam extends SQLMetricParam[LongSQLMetricValue, Long] { +private[spark] object LongSQLMetricParam extends SQLMetricParam[LongSQLMetricValue, Long] { val stringValue : Seq[Long] => String = _.sum.toString @@ -105,7 +105,7 @@ private[sql] object LongSQLMetricParam extends SQLMetricParam[LongSQLMetricValue override def zero: LongSQLMetricValue = new LongSQLMetricValue(0L) } -private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) +private[spark] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) extends SQLMetricParam[LongSQLMetricValue, Long] { override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) @@ -118,7 +118,7 @@ private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, init override def zero: LongSQLMetricValue = new LongSQLMetricValue(initialValue) } -private[sql] object SQLMetrics { +private[spark] object SQLMetrics { private def createLongMetric( sc: SparkContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala similarity index 76% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala index 9c27944d42fc..5e99627dd0b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala @@ -15,15 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.ui +package org.apache.spark.ui.sql import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.Logging import org.apache.spark.ui.{SparkUI, SparkUITab} -private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) - extends SparkUITab(sparkUI, SQLTab.nextTabName) with Logging { +import scala.collection.mutable + +private[spark] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) + extends SparkUITab(sparkUI, SQLTab.nextTabName(sparkUI)) with Logging { val parent = sparkUI @@ -38,9 +40,10 @@ private[sql] object SQLTab { private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" - private val nextTabId = new AtomicInteger(0) + private val nextTabIds = new mutable.HashMap[SparkUI, AtomicInteger] - private def nextTabName: String = { + private def nextTabName(sparkUI: SparkUI): String = { + val nextTabId = nextTabIds.getOrElseUpdate(sparkUI, new AtomicInteger(0)) val nextId = nextTabId.getAndIncrement() if (nextId == 0) "SQL" else s"SQL$nextId" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala index 466681331166..ed2f428c9002 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala @@ -15,17 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.ui +package org.apache.spark.ui.sql import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.sql.SparkPlanInfo - import scala.collection.mutable -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{LongSQLMetricParam, SQLMetricParam, SQLMetricValue} - /** * A graph used for storing information of an executionPlan of DataFrame. * @@ -45,7 +40,7 @@ private[ui] case class SparkPlanGraph( } } -private[sql] object SparkPlanGraph { +private[spark] object SparkPlanGraph { /** * Build a SparkPlanGraph from the root of a SparkPlan tree. diff --git a/core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala index d84befeea812..047c27eab735 100644 --- a/core/src/main/scala/org/apache/spark/sql/SparkPlanInfo.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.ui.sql import org.apache.spark.annotation.DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala b/core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala index 46b35ff156a9..a2de0b5cbadf 100644 --- a/core/src/main/scala/org/apache/spark/sql/SqlMetricInfo.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.ui.sql /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a29c42593e6c..6227b2e2859a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import java.util.{Properties, UUID} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.sql.{SqlMetricInfo, SparkPlanInfo} +import org.apache.spark.ui.sql.{SqlMetricInfo, SparkPlanInfo} import scala.collection.JavaConverters._ import scala.collection.Map diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5e7198f97438..bc02e32c17ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import java.beans.{BeanInfo, Introspector} import java.util.Properties import java.util.concurrent.atomic.AtomicReference +import org.apache.spark.ui.sql.{SQLTab, SQLListener} import scala.collection.JavaConverters._ import scala.collection.immutable @@ -43,7 +44,6 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, _} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.{execution => sparkexecution} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 6f3f1bd97ad5..f79efc01f255 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -18,13 +18,12 @@ package org.apache.spark.sql.execution import java.util.HashMap - import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 8bb293ae87e6..f55c59076d27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.util.concurrent.atomic.AtomicBoolean +import org.apache.spark.ui.sql.{LongSQLMetric, SQLMetric} + import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging @@ -31,7 +33,6 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric} import org.apache.spark.sql.types.DataType object SparkPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index dd66e58284e0..fbe10b5717c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{SqlMetricInfo, SparkPlanInfo} +import org.apache.spark.ui.sql.{SqlMetricInfo, SparkPlanInfo} object SparkPlanInfo { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala index 4d37106e007f..d053278f5ed8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples, Distribution} import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics case class SortBasedAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index 64c673064f57..41b136e025ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression2, AggregateFunction2} -import org.apache.spark.sql.execution.metric.LongSQLMetric +import org.apache.spark.ui.sql.LongSQLMetric /** * An iterator used to evaluate [[AggregateFunction2]]. It assumes the input rows have been diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 0d3a4b36c161..42ae1873ddf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression2 import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{UnsafeFixedWidthAggregationMap, UnaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.StructType +import org.apache.spark.ui.sql.SQLMetrics case class TungstenAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index fb2fc98e34fb..5348e0e21ed4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.aggregate +import org.apache.spark.ui.sql.LongSQLMetric + import scala.collection.mutable.ArrayBuffer import org.apache.spark.unsafe.KVIterator @@ -26,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{UnsafeKVExternalSorter, UnsafeFixedWidthAggregationMap} -import org.apache.spark.sql.execution.metric.LongSQLMetric import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 2bb3dba5bd2b..2ef06a476595 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.encoders.Encoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.MutablePair import org.apache.spark.util.random.PoissonSampler import org.apache.spark.{HashPartitioner, SparkEnv} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 1d381e2eaef3..50435e80715e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.SQLMetrics + import scala.concurrent._ import scala.concurrent.duration._ @@ -25,7 +27,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.ThreadUtils import org.apache.spark.{InternalAccumulator, TaskContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index ab81bd7b3fc0..c73655612ee5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.SQLMetrics + import scala.concurrent._ import scala.concurrent.duration._ @@ -26,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index c5cd6a2fd637..023ccbf0e212 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index efef8c8a8b96..85e015be3fd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala index 0243e196dbc3..d7ad0623dacc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 7ce4a517838c..5d8135153ac2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.LongSQLMetric +import org.apache.spark.ui.sql.LongSQLMetric trait HashJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala index 15b06b1537f8..1673ee94aa29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.LongSQLMetric +import org.apache.spark.ui.sql.LongSQLMetric import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index beb141ade616..3375a2e2bb14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.LongSQLMetric +import org.apache.spark.ui.sql.LongSQLMetric trait HashSemiJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index cc8abb1ba463..953d40559277 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.sql.execution.local.LocalNode -import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} +import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.unsafe.memory.MemoryLocation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index efa7b49410ed..e986c46cff1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics /** * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala index bf3b05be981f..bab2b1d36677 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, Distribution, ClusteredDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 755986af8b95..9987284d8543 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.ui.sql.SQLMetrics /** * Performs an inner hash join of two child relations by first shuffling the data using the join diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala index 6b2cb9d8f689..1481439d371f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.SQLMetrics + import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD @@ -25,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.sql.execution.metric.SQLMetrics /** * Performs a hash based outer join for two child relations by shuffling the data using diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 17030947b7bb..3faf2f1da616 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} + import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -24,7 +26,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} -import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} /** * Performs an sort merge join of two child relations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala index 7e854e6702f7..05f313155ae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} + import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -24,7 +26,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} import org.apache.spark.util.collection.BitSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala index dd92dda48060..be3712c6238f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.StructType +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter import org.apache.spark.{SparkEnv, InternalAccumulator, TaskContext} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala index 475037bd4537..2109cad1845e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala @@ -21,8 +21,8 @@ import org.apache.spark._ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.ui.sql.SQLMetrics class TungstenAggregationIteratorSuite extends SparkFunSuite with SharedSQLContext { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index e5fd9e277fc6..53ef493f6ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 0d798554fa63..50ebbd11954f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -23,17 +23,19 @@ import org.apache.spark.sql.execution.SparkPlanInfo import scala.collection.mutable -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ -import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.ui.sql.{SQLMetrics, SparkPlanGraph} import org.apache.spark.util.Utils + + class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index b839db7eb6fb..87e3db76eff0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -21,11 +21,11 @@ import java.util.Properties import org.apache.spark.{SparkException, SparkContext, SparkConf, SparkFunSuite} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.sql.execution.metric.LongSQLMetricValue import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.ui.sql.{LongSQLMetricValue, SparkPlanGraph, SQLListener} class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index c328734df316..105f5bb32f6f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,6 +23,8 @@ import java.sql.Timestamp import java.util.concurrent.TimeUnit import java.util.regex.Pattern +import org.apache.spark.ui.sql.SQLListener + import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions @@ -47,7 +49,6 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, SqlParser} import org.apache.spark.sql.execution.datasources.{ResolveDataSource, DataSourceStrategy, PreInsertCastAndRename, PreWriteCheck} -import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.execution.{CacheManager, ExecutedCommand, ExtractPythonUDFs, SetCommand} import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} From 3833055b7c94247b700e6fd0565629e71611d307 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 21 Oct 2015 14:32:28 +0800 Subject: [PATCH 04/22] rename SqlMetricInfo class name --- .../ui/sql/{SqlMetricInfo.scala => SQLMetricInfo.scala} | 2 +- .../scala/org/apache/spark/ui/sql/SparkPlanInfo.scala | 2 +- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 8 ++++---- .../org/apache/spark/sql/execution/SparkPlanInfo.scala | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) rename core/src/main/scala/org/apache/spark/ui/sql/{SqlMetricInfo.scala => SQLMetricInfo.scala} (97%) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala rename to core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala index a2de0b5cbadf..9120c54109bc 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SqlMetricInfo.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala @@ -21,7 +21,7 @@ package org.apache.spark.ui.sql * :: DeveloperApi :: * Stores information about a SQL Metric. */ -class SqlMetricInfo( +class SQLMetricInfo( val name: String, val accumulatorId: Long, val metricParam: String) { diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala index 047c27eab735..1788f1f19721 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala @@ -28,6 +28,6 @@ class SparkPlanInfo( val nodeName: String, val simpleString: String, val children: Seq[SparkPlanInfo], - val metrics: Seq[SqlMetricInfo] + val metrics: Seq[SQLMetricInfo] ) { } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 6227b2e2859a..b012177d4262 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import java.util.{Properties, UUID} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.ui.sql.{SqlMetricInfo, SparkPlanInfo} +import org.apache.spark.ui.sql.{SQLMetricInfo, SparkPlanInfo} import scala.collection.JavaConverters._ import scala.collection.Map @@ -458,7 +458,7 @@ private[spark] object JsonProtocol { ("Metrics" -> metrics) } - def sqlMetricInfoToJson(sqlMetricInfo: SqlMetricInfo): JValue = { + def sqlMetricInfoToJson(sqlMetricInfo: SQLMetricInfo): JValue = { ("Name" -> sqlMetricInfo.name) ~ ("Accumulator Id" -> sqlMetricInfo.accumulatorId) ~ ("Metric Param" -> sqlMetricInfo.metricParam) @@ -960,11 +960,11 @@ private[spark] object JsonProtocol { new SparkPlanInfo(nodeName, simpleString, children, metrics) } - def sqlMetricInfoFromJson(json: JValue): SqlMetricInfo = { + def sqlMetricInfoFromJson(json: JValue): SQLMetricInfo = { val name = (json \ "Name").extract[String] val accumulatorId = (json \ "Accumulator Id").extract[Long] val metricParam = (json \ "Metric Param").extract[String] - new SqlMetricInfo(name, accumulatorId, metricParam) + new SQLMetricInfo(name, accumulatorId, metricParam) } /** -------------------------------- * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index fbe10b5717c3..a1eb6e81a400 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.execution -import org.apache.spark.ui.sql.{SqlMetricInfo, SparkPlanInfo} +import org.apache.spark.ui.sql.{SQLMetricInfo, SparkPlanInfo} object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { //Todo correct the param val metrics = plan.metrics.toSeq.map { case (key, metric) => - new SqlMetricInfo(metric.name.getOrElse(key), metric.id, metric.param.toString) + new SQLMetricInfo(metric.name.getOrElse(key), metric.id, metric.param.toString) } val children = plan.children.map(fromSparkPlan) From c0abfc6b7432750812a47790fe51b35dacba7429 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 22 Oct 2015 10:22:51 +0800 Subject: [PATCH 05/22] Update sql metric param --- .../apache/spark/ui/sql/SQLMetricInfo.scala | 3 + .../org/apache/spark/ui/sql/SQLMetrics.scala | 71 +++++++++---------- .../apache/spark/ui/sql/SparkPlanGraph.scala | 6 +- .../spark/sql/execution/SparkPlanInfo.scala | 9 +-- 4 files changed, 44 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala index 9120c54109bc..eee3fd4b1694 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala @@ -17,10 +17,13 @@ package org.apache.spark.ui.sql +import org.apache.spark.annotation.DeveloperApi + /** * :: DeveloperApi :: * Stores information about a SQL Metric. */ +@DeveloperApi class SQLMetricInfo( val name: String, val accumulatorId: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala index 24c8288cd91d..8e6365b3be3b 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala @@ -91,20 +91,6 @@ private[spark] class LongSQLMetric(name: String, param: LongSQLMetricParam) } } -private[spark] object LongSQLMetricParam extends SQLMetricParam[LongSQLMetricValue, Long] { - - val stringValue : Seq[Long] => String = _.sum.toString - - override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) - - override def addInPlace(r1: LongSQLMetricValue, r2: LongSQLMetricValue): LongSQLMetricValue = - r1.add(r2.value) - - override def zero(initialValue: LongSQLMetricValue): LongSQLMetricValue = zero - - override def zero: LongSQLMetricValue = new LongSQLMetricValue(0L) -} - private[spark] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) extends SQLMetricParam[LongSQLMetricValue, Long] { @@ -118,21 +104,39 @@ private[spark] class LongSQLMetricParam(val stringValue: Seq[Long] => String, in override def zero: LongSQLMetricValue = new LongSQLMetricValue(initialValue) } +private[spark] object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) + +private[spark] object StaticsLongSQLMetricParam extends LongSQLMetricParam( + (values: Seq[Long]) => { + // This is a workaround for SPARK-11013. + // We use -1 as initial value of the accumulator, if the accumulator is valid, we will update + // it at the end of task and the value will be at least 0. + val validValues = values.filter(_ >= 0) + val Seq(sum, min, med, max) = { + val metric = if (validValues.length == 0) { + Seq.fill(4)(0L) + } else { + val sorted = validValues.sorted + Seq(sorted.sum, sorted(0), sorted(validValues.length / 2), sorted(validValues.length - 1)) + } + metric.map(Utils.bytesToString) + } + s"\n$sum ($min, $med, $max)" + }, -1L) + private[spark] object SQLMetrics { private def createLongMetric( sc: SparkContext, name: String, - stringValue: Seq[Long] => String, - initialValue: Long): LongSQLMetric = { - val param = new LongSQLMetricParam(stringValue, initialValue) + param: LongSQLMetricParam): LongSQLMetric = { val acc = new LongSQLMetric(name, param) sc.cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc } def createLongMetric(sc: SparkContext, name: String): LongSQLMetric = { - createLongMetric(sc, name, _.sum.toString, 0L) + createLongMetric(sc, name, LongSQLMetricParam) } /** @@ -140,31 +144,22 @@ private[spark] object SQLMetrics { * spill size, etc. */ def createSizeMetric(sc: SparkContext, name: String): LongSQLMetric = { - val stringValue = (values: Seq[Long]) => { - // This is a workaround for SPARK-11013. - // We use -1 as initial value of the accumulator, if the accumulator is valid, we will update - // it at the end of task and the value will be at least 0. - val validValues = values.filter(_ >= 0) - val Seq(sum, min, med, max) = { - val metric = if (validValues.length == 0) { - Seq.fill(4)(0L) - } else { - val sorted = validValues.sorted - Seq(sorted.sum, sorted(0), sorted(validValues.length / 2), sorted(validValues.length - 1)) - } - metric.map(Utils.bytesToString) - } - s"\n$sum ($min, $med, $max)" + createLongMetric(sc, s"$name total (min, med, max)", StaticsLongSQLMetricParam) + } + + def getMetricParam(metricParamName: String): SQLMetricParam[SQLMetricValue[Any], Any] = { + val longSQLMetricParam = Utils.getFormattedClassName(LongSQLMetricParam) + val staticsSQLMetricParam = Utils.getFormattedClassName(StaticsLongSQLMetricParam) + val metricParam = metricParamName match { + case `longSQLMetricParam` => LongSQLMetricParam + case `staticsSQLMetricParam` => StaticsLongSQLMetricParam } - // The final result of this metric in physical operator UI may looks like: - // data size total (min, med, max): - // 100GB (100MB, 1GB, 10GB) - createLongMetric(sc, s"$name total (min, med, max)", stringValue, -1L) + metricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]] } /** * A metric that its value will be ignored. Use this one when we need a metric parameter but don't * care about the value. */ - val nullLongMetric = new LongSQLMetric("null", new LongSQLMetricParam(_.sum.toString, 0L)) + val nullLongMetric = new LongSQLMetric("null", LongSQLMetricParam) } diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala index ed2f428c9002..51b9f518d095 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala @@ -19,6 +19,8 @@ package org.apache.spark.ui.sql import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.util.Utils + import scala.collection.mutable /** @@ -58,11 +60,9 @@ private[spark] object SparkPlanGraph { nodeIdGenerator: AtomicLong, nodes: mutable.ArrayBuffer[SparkPlanGraphNode], edges: mutable.ArrayBuffer[SparkPlanGraphEdge]): SparkPlanGraphNode = { - //TODO correct the metric Param val metrics = planInfo.metrics.map { metric => SQLPlanMetric(metric.name, metric.accumulatorId, - LongSQLMetricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) - // metric.metricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) + SQLMetrics.getMetricParam(metric.metricParam)) } val node = SparkPlanGraphNode( nodeIdGenerator.getAndIncrement(), planInfo.nodeName, planInfo.simpleString, metrics) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index a1eb6e81a400..2bd49742701d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.execution -import org.apache.spark.ui.sql.{SQLMetricInfo, SparkPlanInfo} +import org.apache.spark.ui.sql._ +import org.apache.spark.util.Utils -object SparkPlanInfo { +private[sql] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { - //Todo correct the param val metrics = plan.metrics.toSeq.map { case (key, metric) => - new SQLMetricInfo(metric.name.getOrElse(key), metric.id, metric.param.toString) + new SQLMetricInfo(metric.name.getOrElse(key), metric.id, + Utils.getFormattedClassName(metric.param)) } val children = plan.children.map(fromSparkPlan) From a5b1cf42a4847b2c9c9674ce0d6aa4d332498ca2 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 26 Oct 2015 10:09:32 +0800 Subject: [PATCH 06/22] handle accumulator updates in sql history UI --- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +-- .../org/apache/spark/ui/sql/SQLListener.scala | 36 +++++++++++++------ 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 3bed77ef77a9..636243781327 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -30,7 +30,7 @@ import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.ui.scope.RDDOperationGraphListener -import org.apache.spark.ui.sql.{SQLTab, SQLListener} +import org.apache.spark.ui.sql.{SQLHistoryListener, SQLTab, SQLListener} /** * Top level user interface for a Spark application. @@ -151,7 +151,7 @@ private[spark] object SparkUI { appName: String, basePath: String, startTime: Long): SparkUI = { - val sqlListener = new SQLListener(conf) + val sqlListener = new SQLHistoryListener(conf) listenerBus.addListener(sqlListener) val sparkUI = create( None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala index 73f2bbbbffdb..bec6d8616e62 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.sql -import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} @@ -118,7 +117,8 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { for ((taskId, stageId, stageAttemptID, metrics) <- executorMetricsUpdate.taskMetrics) { - updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, metrics, finishTask = false) + updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, metrics.accumulatorUpdates(), + finishTask = false) } } @@ -140,7 +140,7 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log taskEnd.taskInfo.taskId, taskEnd.stageId, taskEnd.stageAttemptId, - taskEnd.taskMetrics, + taskEnd.taskMetrics.accumulatorUpdates(), finishTask = true) } @@ -148,15 +148,12 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log * Update the accumulator values of a task with the latest metrics for this task. This is called * every time we receive an executor heartbeat or when a task finishes. */ - private def updateTaskAccumulatorValues( + protected def updateTaskAccumulatorValues( taskId: Long, stageId: Int, stageAttemptID: Int, - metrics: TaskMetrics, + accumulatorUpdates: Map[Long, Any], finishTask: Boolean): Unit = { - if (metrics == null) { - return - } _stageIdToStageMetrics.get(stageId) match { case Some(stageMetrics) => @@ -174,9 +171,9 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log case Some(taskMetrics) => if (finishTask) { taskMetrics.finished = true - taskMetrics.accumulatorUpdates = metrics.accumulatorUpdates() + taskMetrics.accumulatorUpdates = accumulatorUpdates } else if (!taskMetrics.finished) { - taskMetrics.accumulatorUpdates = metrics.accumulatorUpdates() + taskMetrics.accumulatorUpdates = accumulatorUpdates } else { // If a task is finished, we should not override with accumulator updates from // heartbeat reports @@ -185,7 +182,7 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log // TODO Now just set attemptId to 0. Should fix here when we can get the attempt // id from SparkListenerExecutorMetricsUpdate stageMetrics.taskIdToMetricUpdates(taskId) = new SQLTaskMetrics( - attemptId = 0, finished = finishTask, metrics.accumulatorUpdates()) + attemptId = 0, finished = finishTask, accumulatorUpdates) } } case None => @@ -292,6 +289,23 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log } +private[spark] class SQLHistoryListener(conf: SparkConf) extends SQLListener(conf) { + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { + // Do nothing + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + updateTaskAccumulatorValues( + taskEnd.taskInfo.taskId, + taskEnd.stageId, + taskEnd.stageAttemptId, + taskEnd.taskInfo.accumulables.map { acc => + (acc.id, new LongSQLMetricValue(acc.update.getOrElse("0").toLong)) + }.toMap, + finishTask = true) + } +} + /** * Represent all necessary data for an execution that will be used in Web UI. */ From 7b30bc736f09600b25772e30636f8a5c19c6db5e Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 26 Oct 2015 14:51:25 +0800 Subject: [PATCH 07/22] Use a single SQL Tab for all SparkContext --- .../scala/org/apache/spark/SparkContext.scala | 7 ++++++ .../org/apache/spark/ui/sql/SQLTab.scala | 11 +------- .../org/apache/spark/sql/SQLContext.scala | 25 ++++++++----------- .../apache/spark/sql/hive/HiveContext.scala | 8 ++---- 4 files changed, 21 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a6857b4c7d88..ed203350855d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.apache.spark.ui.sql.SQLListener + import scala.language.implicitConversions import java.io._ @@ -225,6 +227,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _jars: Seq[String] = _ private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ + private var _sqlListener: SQLListener = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -247,6 +250,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) private[spark] def eventLogDir: Option[URI] = _eventLogDir private[spark] def eventLogCodec: Option[String] = _eventLogCodec + private[spark] def sqlListener: SQLListener = _sqlListener // Generate the random name for a temp folder in external block store. // Add a timestamp as the suffix here to make it more safe @@ -1390,6 +1394,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) + if (listener.isInstanceOf[SQLListener]) { + _sqlListener = listener.asInstanceOf[SQLListener] + } } /** diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala index 5e99627dd0b3..449a570be33c 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} import scala.collection.mutable private[spark] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) - extends SparkUITab(sparkUI, SQLTab.nextTabName(sparkUI)) with Logging { + extends SparkUITab(sparkUI, "SQL") with Logging { val parent = sparkUI @@ -37,14 +37,5 @@ private[spark] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) } private[sql] object SQLTab { - private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" - - private val nextTabIds = new mutable.HashMap[SparkUI, AtomicInteger] - - private def nextTabName(sparkUI: SparkUI): String = { - val nextTabId = nextTabIds.getOrElseUpdate(sparkUI, new AtomicInteger(0)) - val nextId = nextTabId.getAndIncrement() - if (nextId == 0) "SQL" else s"SQL$nextId" - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index bc02e32c17ea..46e3535222ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -68,15 +68,13 @@ import org.apache.spark.util.Utils class SQLContext private[sql]( @transient val sparkContext: SparkContext, @transient protected[sql] val cacheManager: CacheManager, - @transient private[sql] val listener: SQLListener, val isRootContext: Boolean) extends org.apache.spark.Logging with Serializable { self => - def this(sparkContext: SparkContext) = { - this(sparkContext, new CacheManager, SQLContext.createListenerAndUI(sparkContext), true) - } + def this(sparkContext: SparkContext) = this(sparkContext, new CacheManager, true) + def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) // If spark.sql.allowMultipleContexts is true, we will throw an exception if a user @@ -101,6 +99,15 @@ class SQLContext private[sql]( } } + @transient private[sql] val listener: SQLListener = if (sparkContext.sqlListener != null) { + sparkContext.sqlListener + } else { + val listener = new SQLListener(sparkContext.conf) + sparkContext.addSparkListener(listener) + sparkContext.ui.foreach(new SQLTab(listener, _)) + listener + } + /** * Returns a SQLContext as new session, with separated SQL configurations, temporary tables, * registered functions, but sharing the same SparkContext, CacheManager, SQLListener and SQLTab. @@ -111,7 +118,6 @@ class SQLContext private[sql]( new SQLContext( sparkContext = sparkContext, cacheManager = cacheManager, - listener = listener, isRootContext = false) } @@ -1328,13 +1334,4 @@ 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 - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 105f5bb32f6f..565632ccb8b0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,8 +23,6 @@ import java.sql.Timestamp import java.util.concurrent.TimeUnit import java.util.regex.Pattern -import org.apache.spark.ui.sql.SQLListener - import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions @@ -91,15 +89,14 @@ private[hive] case class CurrentDatabase(ctx: HiveContext) class HiveContext private[hive]( sc: SparkContext, cacheManager: CacheManager, - listener: SQLListener, @transient private val execHive: ClientWrapper, @transient private val metaHive: ClientInterface, isRootContext: Boolean) - extends SQLContext(sc, cacheManager, listener, isRootContext) with Logging { + extends SQLContext(sc, cacheManager, isRootContext) with Logging { self => def this(sc: SparkContext) = { - this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), null, null, true) + this(sc, new CacheManager, null, null, true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -116,7 +113,6 @@ class HiveContext private[hive]( new HiveContext( sc = sc, cacheManager = cacheManager, - listener = listener, execHive = executionHive.newSession(), metaHive = metadataHive.newSession(), isRootContext = false) From d52288bb2e18a5f0e898110893a091919e13ea84 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 27 Oct 2015 16:49:30 +0800 Subject: [PATCH 08/22] update style --- .../src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala | 5 ++++- .../main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala | 2 -- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 +-- .../scala/org/apache/spark/sql/execution/Aggregate.scala | 1 + .../scala/org/apache/spark/sql/execution/SparkPlan.scala | 3 +-- .../execution/aggregate/TungstenAggregationIterator.scala | 3 +-- .../spark/sql/execution/joins/BroadcastHashJoin.scala | 3 +-- .../spark/sql/execution/joins/BroadcastHashOuterJoin.scala | 3 +-- .../sql/execution/joins/BroadcastLeftSemiJoinHash.scala | 2 +- .../spark/sql/execution/joins/ShuffledHashOuterJoin.scala | 3 +-- .../apache/spark/sql/execution/joins/SortMergeJoin.scala | 3 +-- .../spark/sql/execution/joins/SortMergeOuterJoin.scala | 3 +-- .../apache/spark/sql/execution/metric/SQLMetricsSuite.scala | 6 +----- 13 files changed, 15 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala index 8e6365b3be3b..b51aa83b35ef 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.ui.sql -import org.apache.spark.{SparkContext, AccumulableParam, Accumulable} import org.apache.spark.util.Utils +import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} /** * Create a layer for specialized metric. We cannot add `@specialized` to @@ -144,6 +144,9 @@ private[spark] object SQLMetrics { * spill size, etc. */ def createSizeMetric(sc: SparkContext, name: String): LongSQLMetric = { + // The final result of this metric in physical operator UI may looks like: + // data size total (min, med, max): + // 100GB (100MB, 1GB, 10GB) createLongMetric(sc, s"$name total (min, med, max)", StaticsLongSQLMetricParam) } diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala index 51b9f518d095..8e43e3bc6dcb 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala @@ -19,8 +19,6 @@ package org.apache.spark.ui.sql import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.util.Utils - import scala.collection.mutable /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 46e3535222ae..035b5ec4b567 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,8 +21,6 @@ import java.beans.{BeanInfo, Introspector} import java.util.Properties import java.util.concurrent.atomic.AtomicReference -import org.apache.spark.ui.sql.{SQLTab, SQLListener} - import scala.collection.JavaConverters._ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag @@ -48,6 +46,7 @@ import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.{execution => sparkexecution} import org.apache.spark.sql.util.ExecutionListenerManager +import org.apache.spark.ui.sql.{SQLTab, SQLListener} import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index f79efc01f255..1fde92429ddc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.util.HashMap + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index f55c59076d27..c3ad8a130daf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution import java.util.concurrent.atomic.AtomicBoolean -import org.apache.spark.ui.sql.{LongSQLMetric, SQLMetric} - import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging @@ -34,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.DataType +import org.apache.spark.ui.sql.{LongSQLMetric, SQLMetric} object SparkPlan { protected[sql] val currentContext = new ThreadLocal[SQLContext]() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 5348e0e21ed4..94609cbdab1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.aggregate -import org.apache.spark.ui.sql.LongSQLMetric - import scala.collection.mutable.ArrayBuffer import org.apache.spark.unsafe.KVIterator @@ -29,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{UnsafeKVExternalSorter, UnsafeFixedWidthAggregationMap} import org.apache.spark.sql.types.StructType +import org.apache.spark.ui.sql.LongSQLMetric /** * An iterator used to evaluate aggregate functions. It operates on [[UnsafeRow]]s. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 50435e80715e..1975eeb524b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.SQLMetrics - import scala.concurrent._ import scala.concurrent.duration._ @@ -27,6 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.ThreadUtils import org.apache.spark.{InternalAccumulator, TaskContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index c73655612ee5..4f07fcf819ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.SQLMetrics - import scala.concurrent._ import scala.concurrent.duration._ @@ -28,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} +import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 023ccbf0e212..b8db6a9fb0db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.ui.sql.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala index 1481439d371f..14ec475646ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.SQLMetrics - import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD @@ -27,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.ui.sql.SQLMetrics /** * Performs a hash based outer join for two child relations by shuffling the data using diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 3faf2f1da616..90153ee5ee33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} - import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -26,6 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} +import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} /** * Performs an sort merge join of two child relations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala index 05f313155ae3..2ff1baf6f72b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} - import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -27,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} +import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} import org.apache.spark.util.collection.BitSet /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 50ebbd11954f..319e0855334c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.metric import java.io.{ByteArrayInputStream, ByteArrayOutputStream} -import org.apache.spark.sql.execution.SparkPlanInfo - import scala.collection.mutable import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ @@ -28,14 +26,12 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.ui.sql.{SQLMetrics, SparkPlanGraph} import org.apache.spark.util.Utils - - - class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ From 7a2acedfc524e5c5887bd783e6fbbe289313306a Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 28 Oct 2015 13:08:23 +0800 Subject: [PATCH 09/22] Throws exception for unknown events --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b012177d4262..df21b5ff471a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import java.io.IOException import java.util.{Properties, UUID} import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -544,6 +545,7 @@ private[spark] object JsonProtocol { case `metricsUpdate` => executorMetricsUpdateFromJson(json) case `sqlExecutionStart` => sqlExecutionStartFromJson(json) case `sqlExecutionEnd` => sqlExecutionEndFromJson(json) + case other => throw new IOException(s"Unknown Event $other") } } From caab0bab0299d4eb985b2e5e68cc5813faac6dfb Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 28 Oct 2015 13:39:07 +0800 Subject: [PATCH 10/22] Fix build error --- core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala index bec6d8616e62..8fd5a7febe3c 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala @@ -224,6 +224,7 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log } } } + case _ => //ignore } } From 0af5afeafe894614e7c1cb83f343db0a0869ad77 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Wed, 28 Oct 2015 14:07:28 +0800 Subject: [PATCH 11/22] Fix style --- core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala index 8fd5a7febe3c..a60d21a2d968 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala @@ -224,7 +224,7 @@ private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Log } } } - case _ => //ignore + case _ => // Ignore } } From 8d565f24ab365482333d4ff2331c72b01d02358a Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 6 Nov 2015 10:46:35 +0800 Subject: [PATCH 12/22] Avoid moving the sql classes --- .../scala/org/apache/spark/SparkContext.scala | 7 -- .../spark/scheduler/SparkListener.scala | 33 +++--- .../scala/org/apache/spark/ui/SparkUI.scala | 10 +- .../org/apache/spark/util/JsonProtocol.scala | 104 +++++------------- ...spark.scheduler.SparkListenerEventRegister | 1 + .../org/apache/spark/sql/SQLContext.scala | 28 +++-- .../spark/sql/execution/Aggregate.scala | 2 +- .../spark/sql/execution/SQLExecution.scala | 3 +- .../spark/sql/execution/SparkPlan.scala | 2 +- .../spark/sql/execution/SparkPlanInfo.scala | 18 ++- .../aggregate/SortBasedAggregate.scala | 2 +- .../SortBasedAggregationIterator.scala | 2 +- .../aggregate/TungstenAggregate.scala | 2 +- .../TungstenAggregationIterator.scala | 3 +- .../spark/sql/execution/basicOperators.scala | 2 +- .../execution/joins/BroadcastHashJoin.scala | 3 +- .../joins/BroadcastHashOuterJoin.scala | 3 +- .../joins/BroadcastLeftSemiJoinHash.scala | 2 +- .../joins/BroadcastNestedLoopJoin.scala | 2 +- .../execution/joins/CartesianProduct.scala | 2 +- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../sql/execution/joins/HashOuterJoin.scala | 2 +- .../sql/execution/joins/HashSemiJoin.scala | 2 +- .../sql/execution/joins/HashedRelation.scala | 2 +- .../sql/execution/joins/LeftSemiJoinBNL.scala | 2 +- .../execution/joins/LeftSemiJoinHash.scala | 2 +- .../execution/joins/ShuffledHashJoin.scala | 2 +- .../joins/ShuffledHashOuterJoin.scala | 3 +- .../sql/execution/joins/SortMergeJoin.scala | 2 +- .../execution/joins/SortMergeOuterJoin.scala | 2 +- .../sql/execution/metric}/SQLMetricInfo.scala | 2 +- .../sql/execution/metric}/SQLMetrics.scala | 2 +- .../org/apache/spark/sql/execution/sort.scala | 2 +- .../sql/execution/ui}/AllExecutionsPage.scala | 2 +- .../sql/execution/ui}/ExecutionPage.scala | 2 +- .../sql/execution/ui/SQLEventRegister.scala | 34 +++--- .../spark/sql/execution/ui}/SQLListener.scala | 19 +++- .../spark/sql/execution/ui}/SQLTab.scala | 8 +- .../sql/execution/ui}/SparkPlanGraph.scala | 7 +- .../TungstenAggregationIteratorSuite.scala | 2 +- .../execution/joins/HashedRelationSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 5 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 7 +- 44 files changed, 168 insertions(+), 180 deletions(-) create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/metric}/SQLMetricInfo.scala (95%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/metric}/SQLMetrics.scala (99%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/ui}/AllExecutionsPage.scala (99%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/ui}/ExecutionPage.scala (99%) rename core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala => sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala (55%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/ui}/SQLListener.scala (96%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/ui}/SQLTab.scala (86%) rename {core/src/main/scala/org/apache/spark/ui/sql => sql/core/src/main/scala/org/apache/spark/sql/execution/ui}/SparkPlanGraph.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ed203350855d..a6857b4c7d88 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,8 +17,6 @@ package org.apache.spark -import org.apache.spark.ui.sql.SQLListener - import scala.language.implicitConversions import java.io._ @@ -227,7 +225,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _jars: Seq[String] = _ private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ - private var _sqlListener: SQLListener = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -250,7 +247,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) private[spark] def eventLogDir: Option[URI] = _eventLogDir private[spark] def eventLogCodec: Option[String] = _eventLogCodec - private[spark] def sqlListener: SQLListener = _sqlListener // Generate the random name for a temp folder in external block store. // Add a timestamp as the suffix here to make it more safe @@ -1394,9 +1390,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) - if (listener.isInstanceOf[SQLListener]) { - _sqlListener = listener.asInstanceOf[SQLListener] - } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index b9e067af7924..8bc67b74a677 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.ui.sql.SparkPlanInfo +import org.apache.spark.ui.SparkUI import scala.collection.Map import scala.collection.mutable @@ -29,17 +29,20 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} -import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.util.{JsonProtocol, Distribution, Utils} @DeveloperApi -sealed trait SparkListenerEvent +trait SparkListenerEvent { +} @DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) - extends SparkListenerEvent + extends SparkListenerEvent { +} + @DeveloperApi -case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent {} @DeveloperApi case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: TaskInfo) @@ -126,26 +129,18 @@ case class SparkListenerApplicationStart( @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent -@DeveloperApi -case class SparkListenerSQLExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - sparkPlanInfo: SparkPlanInfo, - time: Long) - extends SparkListenerEvent - -@DeveloperApi -case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent - /** * An internal class that describes the metadata of an event log. * This event is not meant to be posted to listeners downstream. */ private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent +trait SparkListenerEventRegister { + def getEventClasses(): List[Class[_]] + def getListener(): SparkListener + def attachUITab(listener: SparkListener, sparkUI: SparkUI) +} + /** * :: DeveloperApi :: * Interface for listening to events from the Spark scheduler. Note that this is an internal diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 636243781327..30780118ef7b 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,6 +21,7 @@ import java.util.Date import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, UIRoot} +import org.apache.spark.util.JsonProtocol import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -30,7 +31,6 @@ import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.ui.scope.RDDOperationGraphListener -import org.apache.spark.ui.sql.{SQLHistoryListener, SQLTab, SQLListener} /** * Top level user interface for a Spark application. @@ -151,11 +151,13 @@ private[spark] object SparkUI { appName: String, basePath: String, startTime: Long): SparkUI = { - val sqlListener = new SQLHistoryListener(conf) - listenerBus.addListener(sqlListener) val sparkUI = create( None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) - new SQLTab(sqlListener, sparkUI) + JsonProtocol.eventRegisters.foreach { eventRegister => + val listener = eventRegister.getListener() + listenerBus.addListener(listener) + eventRegister.attachUITab(listener, sparkUI) + } sparkUI } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index df21b5ff471a..d07a0578a52c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -17,18 +17,19 @@ package org.apache.spark.util -import java.io.IOException -import java.util.{Properties, UUID} +import java.util.{ServiceLoader, Properties, UUID} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.ui.sql.{SQLMetricInfo, SparkPlanInfo} import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.mutable.ListBuffer -import org.json4s.DefaultFormats +import org.json4s.{ShortTypeHints, DefaultFormats} import org.json4s.JsonDSL._ import org.json4s.JsonAST._ +import org.json4s.jackson.JsonMethods._ +import org.json4s.jackson.Serialization.{read, write} import org.apache.spark._ import org.apache.spark.executor._ @@ -51,10 +52,24 @@ import org.apache.spark.storage._ * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields * in `*FromJson` methods. */ -private[spark] object JsonProtocol { +private[spark] object JsonProtocol extends Logging { // TODO: Remove this file and put JSON serialization into each individual class. - private implicit val format = DefaultFormats + val eventRegisters: Iterable[SparkListenerEventRegister] = { + val loader = Utils.getContextOrSparkClassLoader + val serviceLoader = ServiceLoader.load(classOf[SparkListenerEventRegister], loader) + serviceLoader.asScala + } + + var eventClasses = new ListBuffer[Class[_]]() + eventRegisters.foreach { eventRegister => + eventClasses ++= eventRegister.getEventClasses() + } + + private implicit val format = new DefaultFormats { + override val typeHintFieldName = "Event" + override val typeHints = ShortTypeHints(eventClasses.toList) + } /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | @@ -96,12 +111,9 @@ private[spark] object JsonProtocol { logStartToJson(logStart) case metricsUpdate: SparkListenerExecutorMetricsUpdate => executorMetricsUpdateToJson(metricsUpdate) - case sqlExecutionStart: SparkListenerSQLExecutionStart => - sqlExecutionStartToJson(sqlExecutionStart) - case sqlExecutionEnd: SparkListenerSQLExecutionEnd => - sqlExecutionEndToJson(sqlExecutionEnd) case blockUpdated: SparkListenerBlockUpdated => throw new MatchError(blockUpdated) // TODO(ekl) implement this + case _ => parse(write(event)) } } @@ -245,22 +257,6 @@ private[spark] object JsonProtocol { }) } - def sqlExecutionStartToJson(sqlExecutionStart: SparkListenerSQLExecutionStart): JValue = { - ("Event" -> Utils.getFormattedClassName(sqlExecutionStart)) ~ - ("Execution ID" -> sqlExecutionStart.executionId) ~ - ("Description" -> sqlExecutionStart.description) ~ - ("Details" -> sqlExecutionStart.details) ~ - ("Physical Plan Description" -> sqlExecutionStart.physicalPlanDescription) ~ - ("SparkPlan Info" -> sparkPlanInfoToJson(sqlExecutionStart.sparkPlanInfo)) ~ - ("Timestamp" -> sqlExecutionStart.time) - } - - def sqlExecutionEndToJson(sqlExecutionEnd: SparkListenerSQLExecutionEnd): JValue = { - ("Event" -> Utils.getFormattedClassName(sqlExecutionEnd)) ~ - ("Execution ID" -> sqlExecutionEnd.executionId) ~ - ("Timestamp" -> sqlExecutionEnd.time) - } - /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | * -------------------------------------------------------------------- */ @@ -450,21 +446,6 @@ private[spark] object JsonProtocol { ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) } - def sparkPlanInfoToJson(planInfo: SparkPlanInfo): JValue = { - val children = JArray(planInfo.children.map(sparkPlanInfoToJson).toList) - val metrics = JArray(planInfo.metrics.map(sqlMetricInfoToJson).toList) - ("Node Name" -> planInfo.nodeName) ~ - ("Simple String" -> planInfo.simpleString) ~ - ("Children" -> children) ~ - ("Metrics" -> metrics) - } - - def sqlMetricInfoToJson(sqlMetricInfo: SQLMetricInfo): JValue = { - ("Name" -> sqlMetricInfo.name) ~ - ("Accumulator Id" -> sqlMetricInfo.accumulatorId) ~ - ("Metric Param" -> sqlMetricInfo.metricParam) - } - /** ------------------------------ * * Util JSON serialization methods | * ------------------------------- */ @@ -522,10 +503,9 @@ private[spark] object JsonProtocol { val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) val logStart = Utils.getFormattedClassName(SparkListenerLogStart) val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) - val sqlExecutionStart = Utils.getFormattedClassName(SparkListenerSQLExecutionStart) - val sqlExecutionEnd = Utils.getFormattedClassName(SparkListenerSQLExecutionEnd) - (json \ "Event").extract[String] match { + val event = (json \ "Event").extract[String] + event match { case `stageSubmitted` => stageSubmittedFromJson(json) case `stageCompleted` => stageCompletedFromJson(json) case `taskStart` => taskStartFromJson(json) @@ -543,29 +523,10 @@ private[spark] object JsonProtocol { case `executorRemoved` => executorRemovedFromJson(json) case `logStart` => logStartFromJson(json) case `metricsUpdate` => executorMetricsUpdateFromJson(json) - case `sqlExecutionStart` => sqlExecutionStartFromJson(json) - case `sqlExecutionEnd` => sqlExecutionEndFromJson(json) - case other => throw new IOException(s"Unknown Event $other") + case other => read[SparkListenerEvent](compact(render(json))) } } - def sqlExecutionStartFromJson(json: JValue): SparkListenerSQLExecutionStart = { - val executionId = (json \ "Execution ID").extract[Long] - val description = (json \ "Description").extract[String] - val details = (json \ "Details").extract[String] - val physicalPlanDescription = (json \ "Physical Plan Description").extract[String] - val sparkPlanInfo = sparkPlanInfoFromJson(json \ "SparkPlan Info") - val time = (json \ "Timestamp").extract[Long] - SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, - sparkPlanInfo, time) - } - - def sqlExecutionEndFromJson(json: JValue): SparkListenerSQLExecutionEnd = { - val executionId = (json \ "Execution ID").extract[Long] - val time = (json \ "Timestamp").extract[Long] - SparkListenerSQLExecutionEnd(executionId, time) - } - def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { val stageInfo = stageInfoFromJson(json \ "Stage Info") val properties = propertiesFromJson(json \ "Properties") @@ -954,21 +915,6 @@ private[spark] object JsonProtocol { new ExecutorInfo(executorHost, totalCores, logUrls) } - def sparkPlanInfoFromJson(json: JValue): SparkPlanInfo = { - val nodeName = (json \ "Node Name").extract[String] - val simpleString = (json \ "Simple String").extract[String] - val children = (json \ "Children").extract[List[JValue]].map(sparkPlanInfoFromJson) - val metrics = (json \ "Metrics").extract[List[JValue]].map(sqlMetricInfoFromJson) - new SparkPlanInfo(nodeName, simpleString, children, metrics) - } - - def sqlMetricInfoFromJson(json: JValue): SQLMetricInfo = { - val name = (json \ "Name").extract[String] - val accumulatorId = (json \ "Accumulator Id").extract[Long] - val metricParam = (json \ "Metric Param").extract[String] - new SQLMetricInfo(name, accumulatorId, metricParam) - } - /** -------------------------------- * * Util JSON deserialization methods | * --------------------------------- */ diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister new file mode 100644 index 000000000000..e9e60b6260e0 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister @@ -0,0 +1 @@ +org.apache.spark.sql.execution.ui.SQLEventRegister diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 035b5ec4b567..c10bcca51d61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -42,11 +42,11 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, _} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.{execution => sparkexecution} import org.apache.spark.sql.util.ExecutionListenerManager -import org.apache.spark.ui.sql.{SQLTab, SQLListener} import org.apache.spark.util.Utils /** @@ -67,12 +67,15 @@ import org.apache.spark.util.Utils class SQLContext private[sql]( @transient val sparkContext: SparkContext, @transient protected[sql] val cacheManager: CacheManager, + @transient private[sql] val listener: SQLListener, val isRootContext: Boolean) extends org.apache.spark.Logging with Serializable { self => - def this(sparkContext: SparkContext) = this(sparkContext, new CacheManager, true) + def this(sparkContext: SparkContext) = { + this(sparkContext, new CacheManager, SQLContext.createListenerAndUI(sparkContext), true) + } def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) @@ -98,15 +101,6 @@ class SQLContext private[sql]( } } - @transient private[sql] val listener: SQLListener = if (sparkContext.sqlListener != null) { - sparkContext.sqlListener - } else { - val listener = new SQLListener(sparkContext.conf) - sparkContext.addSparkListener(listener) - sparkContext.ui.foreach(new SQLTab(listener, _)) - listener - } - /** * Returns a SQLContext as new session, with separated SQL configurations, temporary tables, * registered functions, but sharing the same SparkContext, CacheManager, SQLListener and SQLTab. @@ -117,6 +111,7 @@ class SQLContext private[sql]( new SQLContext( sparkContext = sparkContext, cacheManager = cacheManager, + listener = listener, isRootContext = false) } @@ -1250,6 +1245,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. * @@ -1333,4 +1330,13 @@ object SQLContext { } } + private[sql] def createListenerAndUI(sc: SparkContext): SQLListener = { + if (sqlListener.get() == null) { + val listener = new SQLListener(sc.conf) + sqlListener.compareAndSet(null, listener) + sc.addSparkListener(listener) + sc.ui.foreach(new SQLTab(listener, _)) + } + sqlListener.get() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 1fde92429ddc..6f3f1bd97ad5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.ui.sql.SQLMetrics +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 8152ec744144..34971986261c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution import java.util.concurrent.atomic.AtomicLong import org.apache.spark.SparkContext -import org.apache.spark.scheduler.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionStart, + SparkListenerSQLExecutionEnd} import org.apache.spark.util.Utils private[sql] object SQLExecution { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index c3ad8a130daf..8bb293ae87e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -31,8 +31,8 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric} import org.apache.spark.sql.types.DataType -import org.apache.spark.ui.sql.{LongSQLMetric, SQLMetric} object SparkPlan { protected[sql] val currentContext = new ThreadLocal[SQLContext]() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 2bd49742701d..d8d65e51a704 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -17,12 +17,26 @@ package org.apache.spark.sql.execution -import org.apache.spark.ui.sql._ +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 = { + 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)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala index d053278f5ed8..13add105cea7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples, Distribution} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} -import org.apache.spark.ui.sql.SQLMetrics case class SortBasedAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index 41b136e025ad..64c673064f57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression2, AggregateFunction2} -import org.apache.spark.ui.sql.LongSQLMetric +import org.apache.spark.sql.execution.metric.LongSQLMetric /** * An iterator used to evaluate [[AggregateFunction2]]. It assumes the input rows have been diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 42ae1873ddf8..c8dc943bbea7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression2 import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{UnsafeFixedWidthAggregationMap, UnaryNode, SparkPlan} import org.apache.spark.sql.types.StructType -import org.apache.spark.ui.sql.SQLMetrics case class TungstenAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 94609cbdab1a..c7b3e2ce9faa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.aggregate +import org.apache.spark.sql.execution.metric.LongSQLMetric + import scala.collection.mutable.ArrayBuffer import org.apache.spark.unsafe.KVIterator @@ -27,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{UnsafeKVExternalSorter, UnsafeFixedWidthAggregationMap} import org.apache.spark.sql.types.StructType -import org.apache.spark.ui.sql.LongSQLMetric /** * An iterator used to evaluate aggregate functions. It operates on [[UnsafeRow]]s. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 2ef06a476595..2bb3dba5bd2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.encoders.Encoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.ui.sql.SQLMetrics +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.MutablePair import org.apache.spark.util.random.PoissonSampler import org.apache.spark.{HashPartitioner, SparkEnv} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 1975eeb524b1..5ee7b3f2972c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.execution.metric.SQLMetrics + import scala.concurrent._ import scala.concurrent.duration._ @@ -25,7 +27,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.ThreadUtils import org.apache.spark.{InternalAccumulator, TaskContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index 4f07fcf819ff..599bcd848089 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.execution.metric.SQLMetrics + import scala.concurrent._ import scala.concurrent.duration._ @@ -26,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index b8db6a9fb0db..fd044a6bb2dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 85e015be3fd2..3f9ddcae7c4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala index d7ad0623dacc..96ddeee1a6c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 5d8135153ac2..7ce4a517838c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.ui.sql.LongSQLMetric +import org.apache.spark.sql.execution.metric.LongSQLMetric trait HashJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala index 1673ee94aa29..15b06b1537f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.ui.sql.LongSQLMetric +import org.apache.spark.sql.execution.metric.LongSQLMetric import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 3375a2e2bb14..beb141ade616 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.ui.sql.LongSQLMetric +import org.apache.spark.sql.execution.metric.LongSQLMetric trait HashSemiJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 953d40559277..bb81469ce632 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.sql.execution.local.LocalNode -import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} +import org.apache.spark.sql.execution.metric.{SQLMetrics, LongSQLMetric} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.unsafe.memory.MemoryLocation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index e986c46cff1e..b23c6ae381d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics /** * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala index bab2b1d36677..6e1299c22a98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, Distribution, ClusteredDistribution} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 9987284d8543..923d28e0a80f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics /** * Performs an inner hash join of two child relations by first shuffling the data using the join diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala index 14ec475646ff..7afa108915a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.sql.execution.metric.SQLMetrics + import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD @@ -25,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} -import org.apache.spark.ui.sql.SQLMetrics /** * Performs a hash based outer join for two child relations by shuffling the data using diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 90153ee5ee33..fe3a6d77523f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} -import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} +import org.apache.spark.sql.execution.metric.{SQLMetrics, LongSQLMetric} /** * Performs an sort merge join of two child relations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala index 2ff1baf6f72b..7e854e6702f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeOuterJoin.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} -import org.apache.spark.ui.sql.{SQLMetrics, LongSQLMetric} import org.apache.spark.util.collection.BitSet /** diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala index eee3fd4b1694..6f1c9f6a3d39 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetricInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.metric import org.apache.spark.annotation.DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index b51aa83b35ef..7c078fcfeb42 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.metric import org.apache.spark.util.Utils import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala index be3712c6238f..dd92dda48060 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/sort.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.StructType -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter import org.apache.spark.{SparkEnv, InternalAccumulator, TaskContext} diff --git a/core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 64b4031a4972..ffe4af89cba2 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.ui import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 0e3fbbebb5af..47d0f34b1b07 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.ui import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala similarity index 55% rename from core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala index 1788f1f19721..ec5fcff82aa5 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala @@ -15,19 +15,25 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.ui -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListenerEventRegister, SparkListener} +import org.apache.spark.ui.SparkUI -/** - * :: DeveloperApi :: - * Stores information about a SQL SparkPlan. - */ -@DeveloperApi -class SparkPlanInfo( - val nodeName: String, - val simpleString: String, - val children: Seq[SparkPlanInfo], - val metrics: Seq[SQLMetricInfo] -) { -} +class SQLEventRegister extends SparkListenerEventRegister { + + override def getEventClasses(): List[Class[_]] = { + List( + classOf[SparkListenerSQLExecutionStart], + classOf[SparkListenerSQLExecutionEnd]) + } + + override def getListener(): SparkListener = { + new SQLHistoryListener(new SparkConf()) + } + + override def attachUITab(listener: SparkListener, sparkUI: SparkUI): Unit = { + new SQLTab(listener.asInstanceOf[SQLListener], sparkUI) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index a60d21a2d968..eedcdb2eed55 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -15,13 +15,30 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.ui +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.SparkPlanInfo +import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetricValue, SQLMetricParam} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} import scala.collection.mutable +@DeveloperApi +case class SparkListenerSQLExecutionStart( + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo, + time: Long) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) + extends SparkListenerEvent + private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Logging { val EXECUTION_ID_KEY = "spark.sql.execution.id" diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala similarity index 86% rename from core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala index 449a570be33c..4f50b2ecdc8f 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SQLTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala @@ -15,16 +15,12 @@ * limitations under the License. */ -package org.apache.spark.ui.sql - -import java.util.concurrent.atomic.AtomicInteger +package org.apache.spark.sql.execution.ui import org.apache.spark.Logging import org.apache.spark.ui.{SparkUI, SparkUITab} -import scala.collection.mutable - -private[spark] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) +private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) extends SparkUITab(sparkUI, "SQL") with Logging { val parent = sparkUI diff --git a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 8e43e3bc6dcb..7af0ff09c5c6 100644 --- a/core/src/main/scala/org/apache/spark/ui/sql/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.spark.ui.sql +package org.apache.spark.sql.execution.ui import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable +import org.apache.spark.sql.execution.SparkPlanInfo +import org.apache.spark.sql.execution.metric.SQLMetrics + /** * A graph used for storing information of an executionPlan of DataFrame. * @@ -40,7 +43,7 @@ private[ui] case class SparkPlanGraph( } } -private[spark] object SparkPlanGraph { +private[sql] object SparkPlanGraph { /** * Build a SparkPlanGraph from the root of a SparkPlan tree. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala index 2109cad1845e..475037bd4537 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala @@ -21,8 +21,8 @@ import org.apache.spark._ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.ui.sql.SQLMetrics class TungstenAggregationIteratorSuite extends SparkFunSuite with SharedSQLContext { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 53ef493f6ec9..e5fd9e277fc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -import org.apache.spark.ui.sql.SQLMetrics import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 319e0855334c..2299cb8f85b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -21,17 +21,18 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.execution.SparkPlanInfo +import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.ui.sql.{SQLMetrics, SparkPlanGraph} import org.apache.spark.util.Utils + class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 87e3db76eff0..f93d081d0c30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.metric.LongSQLMetricValue import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.ui.sql.{LongSQLMetricValue, SparkPlanGraph, SQLListener} class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 565632ccb8b0..c328734df316 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -47,6 +47,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, SqlParser} import org.apache.spark.sql.execution.datasources.{ResolveDataSource, DataSourceStrategy, PreInsertCastAndRename, PreWriteCheck} +import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.execution.{CacheManager, ExecutedCommand, ExtractPythonUDFs, SetCommand} import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} @@ -89,14 +90,15 @@ private[hive] case class CurrentDatabase(ctx: HiveContext) class HiveContext private[hive]( sc: SparkContext, cacheManager: CacheManager, + listener: SQLListener, @transient private val execHive: ClientWrapper, @transient private val metaHive: ClientInterface, isRootContext: Boolean) - extends SQLContext(sc, cacheManager, isRootContext) with Logging { + extends SQLContext(sc, cacheManager, listener, isRootContext) with Logging { self => def this(sc: SparkContext) = { - this(sc, new CacheManager, null, null, true) + this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), null, null, true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -113,6 +115,7 @@ class HiveContext private[hive]( new HiveContext( sc = sc, cacheManager = cacheManager, + listener = listener, execHive = executionHive.newSession(), metaHive = metadataHive.newSession(), isRootContext = false) From 1954d71d674dc35659cba6f9e3b56d42a756778e Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 6 Nov 2015 14:15:59 +0800 Subject: [PATCH 13/22] code clean --- .rat-excludes | 1 + .../spark/scheduler/SparkListener.scala | 21 ++++++++-- .../org/apache/spark/util/JsonProtocol.scala | 10 ++--- .../org/apache/spark/sql/SQLContext.scala | 4 +- .../aggregate/SortBasedAggregate.scala | 2 +- .../aggregate/TungstenAggregate.scala | 2 +- .../TungstenAggregationIterator.scala | 3 +- .../execution/joins/BroadcastHashJoin.scala | 3 +- .../joins/BroadcastHashOuterJoin.scala | 3 +- .../joins/BroadcastLeftSemiJoinHash.scala | 2 +- .../joins/BroadcastNestedLoopJoin.scala | 2 +- .../execution/joins/CartesianProduct.scala | 2 +- .../sql/execution/joins/HashedRelation.scala | 2 +- .../sql/execution/joins/LeftSemiJoinBNL.scala | 2 +- .../execution/joins/LeftSemiJoinHash.scala | 2 +- .../execution/joins/ShuffledHashJoin.scala | 2 +- .../joins/ShuffledHashOuterJoin.scala | 3 +- .../sql/execution/joins/SortMergeJoin.scala | 2 +- .../sql/execution/metric/SQLMetrics.scala | 18 ++++----- .../sql/execution/ui/AllExecutionsPage.scala | 7 ++-- .../sql/execution/ui/ExecutionPage.scala | 6 +-- .../sql/execution/ui/SQLEventRegister.scala | 39 ------------------- .../spark/sql/execution/ui/SQLListener.scala | 24 ++++++++++-- 23 files changed, 77 insertions(+), 85 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala diff --git a/.rat-excludes b/.rat-excludes index 08fba6d351d6..02f7ec606cf8 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -82,4 +82,5 @@ INDEX gen-java.* .*avpr org.apache.spark.sql.sources.DataSourceRegister +org.apache.spark.scheduler.SparkListenerEventRegister .*parquet diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 8bc67b74a677..336314047463 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,8 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.ui.SparkUI - import scala.collection.Map import scala.collection.mutable @@ -29,7 +27,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} -import org.apache.spark.util.{JsonProtocol, Distribution, Utils} +import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.ui.SparkUI @DeveloperApi trait SparkListenerEvent { @@ -42,7 +41,7 @@ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Propert @DeveloperApi -case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent {} +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent @DeveloperApi case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: TaskInfo) @@ -135,9 +134,23 @@ case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent */ private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent +/** + * Interface for registering events defined in other modules like SQL. + */ trait SparkListenerEventRegister { + /** + * Get the class of events to register. + */ def getEventClasses(): List[Class[_]] + + /** + * Get a listener used when rebuilding the history UI. + */ def getListener(): SparkListener + + /** + * Attach a UI Tab to the spark UI if necessary. + */ def attachUITab(listener: SparkListener, sparkUI: SparkUI) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d07a0578a52c..52f6e0c67c27 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -52,13 +52,14 @@ import org.apache.spark.storage._ * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields * in `*FromJson` methods. */ -private[spark] object JsonProtocol extends Logging { +private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. + // Events defined in other modules are allowed to be registered through implementing the + // SparkListenerEventRegister trait. These events are written to Json using Jackson. val eventRegisters: Iterable[SparkListenerEventRegister] = { val loader = Utils.getContextOrSparkClassLoader - val serviceLoader = ServiceLoader.load(classOf[SparkListenerEventRegister], loader) - serviceLoader.asScala + ServiceLoader.load(classOf[SparkListenerEventRegister], loader).asScala } var eventClasses = new ListBuffer[Class[_]]() @@ -504,8 +505,7 @@ private[spark] object JsonProtocol extends Logging { val logStart = Utils.getFormattedClassName(SparkListenerLogStart) val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) - val event = (json \ "Event").extract[String] - event match { + (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) case `stageCompleted` => stageCompletedFromJson(json) case `taskStart` => taskStartFromJson(json) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c10bcca51d61..036fb15d9903 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -76,7 +76,6 @@ class SQLContext private[sql]( def this(sparkContext: SparkContext) = { this(sparkContext, new CacheManager, SQLContext.createListenerAndUI(sparkContext), true) } - def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) // If spark.sql.allowMultipleContexts is true, we will throw an exception if a user @@ -1330,6 +1329,9 @@ 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 = { if (sqlListener.get() == null) { val listener = new SQLListener(sc.conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala index 13add105cea7..4d37106e007f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregate.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples, Distribution} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} +import org.apache.spark.sql.execution.metric.SQLMetrics case class SortBasedAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index c8dc943bbea7..0d3a4b36c161 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression2 import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{UnsafeFixedWidthAggregationMap, UnaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.StructType case class TungstenAggregate( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index c7b3e2ce9faa..fb2fc98e34fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.aggregate -import org.apache.spark.sql.execution.metric.LongSQLMetric - import scala.collection.mutable.ArrayBuffer import org.apache.spark.unsafe.KVIterator @@ -28,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{UnsafeKVExternalSorter, UnsafeFixedWidthAggregationMap} +import org.apache.spark.sql.execution.metric.LongSQLMetric import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 5ee7b3f2972c..1d381e2eaef3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.execution.metric.SQLMetrics - import scala.concurrent._ import scala.concurrent.duration._ @@ -27,6 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.ThreadUtils import org.apache.spark.{InternalAccumulator, TaskContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index 599bcd848089..ab81bd7b3fc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.execution.metric.SQLMetrics - import scala.concurrent._ import scala.concurrent.duration._ @@ -28,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index fd044a6bb2dc..c5cd6a2fd637 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.{InternalAccumulator, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 3f9ddcae7c4f..efef8c8a8b96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.collection.CompactBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala index 96ddeee1a6c1..0243e196dbc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index bb81469ce632..cc8abb1ba463 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.sql.execution.local.LocalNode -import org.apache.spark.sql.execution.metric.{SQLMetrics, LongSQLMetric} +import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.unsafe.memory.MemoryLocation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index b23c6ae381d2..efa7b49410ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala index 6e1299c22a98..bf3b05be981f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, Distribution, ClusteredDistribution} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Build the right table's join keys into a HashSet, and iteratively go through the left diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 923d28e0a80f..755986af8b95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -21,8 +21,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Performs an inner hash join of two child relations by first shuffling the data using the join diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala index 7afa108915a8..6b2cb9d8f689 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.execution.metric.SQLMetrics - import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD @@ -27,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.metric.SQLMetrics /** * Performs a hash based outer join for two child relations by shuffling the data using diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index fe3a6d77523f..17030947b7bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, RowIterator, SparkPlan} -import org.apache.spark.sql.execution.metric.{SQLMetrics, LongSQLMetric} +import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} /** * Performs an sort merge join of two child relations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 7c078fcfeb42..0a42824d55c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -26,7 +26,7 @@ import org.apache.spark.{Accumulable, AccumulableParam, SparkContext} * * An implementation of SQLMetric should override `+=` and `add` to avoid boxing. */ -private[spark] abstract class SQLMetric[R <: SQLMetricValue[T], T]( +private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T]( name: String, val param: SQLMetricParam[R, T]) extends Accumulable[R, T](param.zero, param, Some(name), true) { @@ -39,7 +39,7 @@ private[spark] abstract class SQLMetric[R <: SQLMetricValue[T], T]( * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. */ -private[spark] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends AccumulableParam[R, T] { +private[sql] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends AccumulableParam[R, T] { /** * A function that defines how we aggregate the final accumulator results among all tasks, @@ -54,7 +54,7 @@ private[spark] trait SQLMetricParam[R <: SQLMetricValue[T], T] extends Accumulab * Create a layer for specialized metric. We cannot add `@specialized` to * `Accumulable/AccumulableParam` because it will break Java source compatibility. */ -private[spark] trait SQLMetricValue[T] extends Serializable { +private[sql] trait SQLMetricValue[T] extends Serializable { def value: T @@ -64,7 +64,7 @@ private[spark] trait SQLMetricValue[T] extends Serializable { /** * A wrapper of Long to avoid boxing and unboxing when using Accumulator */ -private[spark] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { +private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetricValue[Long] { def add(incr: Long): LongSQLMetricValue = { _value += incr @@ -79,7 +79,7 @@ private[spark] class LongSQLMetricValue(private var _value : Long) extends SQLMe * A specialized long Accumulable to avoid boxing and unboxing when using Accumulator's * `+=` and `add`. */ -private[spark] class LongSQLMetric(name: String, param: LongSQLMetricParam) +private[sql] class LongSQLMetric(name: String, param: LongSQLMetricParam) extends SQLMetric[LongSQLMetricValue, Long](name, param) { override def +=(term: Long): Unit = { @@ -91,7 +91,7 @@ private[spark] class LongSQLMetric(name: String, param: LongSQLMetricParam) } } -private[spark] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) +private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) extends SQLMetricParam[LongSQLMetricValue, Long] { override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) @@ -104,9 +104,9 @@ private[spark] class LongSQLMetricParam(val stringValue: Seq[Long] => String, in override def zero: LongSQLMetricValue = new LongSQLMetricValue(initialValue) } -private[spark] object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) +private[sql] object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) -private[spark] object StaticsLongSQLMetricParam extends LongSQLMetricParam( +private[sql] object StaticsLongSQLMetricParam extends LongSQLMetricParam( (values: Seq[Long]) => { // This is a workaround for SPARK-11013. // We use -1 as initial value of the accumulator, if the accumulator is valid, we will update @@ -124,7 +124,7 @@ private[spark] object StaticsLongSQLMetricParam extends LongSQLMetricParam( s"\n$sum ($min, $med, $max)" }, -1L) -private[spark] object SQLMetrics { +private[sql] object SQLMetrics { private def createLongMetric( sc: SparkContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index ffe4af89cba2..49646a99d68c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql.execution.ui import javax.servlet.http.HttpServletRequest +import scala.collection.mutable +import scala.xml.Node + import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} -import scala.collection.mutable -import scala.xml.Node - private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with Logging { private val listener = parent.listener diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 47d0f34b1b07..c74ad4040699 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.ui import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import org.apache.spark.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} -import scala.xml.Node - -private[ui] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { +private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { private val listener = parent.listener diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala deleted file mode 100644 index ec5fcff82aa5..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLEventRegister.scala +++ /dev/null @@ -1,39 +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.{SparkListenerEventRegister, SparkListener} -import org.apache.spark.ui.SparkUI - -class SQLEventRegister extends SparkListenerEventRegister { - - override def getEventClasses(): List[Class[_]] = { - List( - classOf[SparkListenerSQLExecutionStart], - classOf[SparkListenerSQLExecutionEnd]) - } - - override def getListener(): SparkListener = { - new SQLHistoryListener(new SparkConf()) - } - - override def attachUITab(listener: SparkListener, sparkUI: SparkUI): Unit = { - new SQLTab(listener.asInstanceOf[SQLListener], sparkUI) - } -} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index eedcdb2eed55..1f63b51f3f4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution.ui +import scala.collection.mutable + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetricValue, SQLMetricParam} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} - -import scala.collection.mutable +import org.apache.spark.ui.SparkUI @DeveloperApi case class SparkListenerSQLExecutionStart( @@ -39,7 +40,24 @@ case class SparkListenerSQLExecutionStart( case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) extends SparkListenerEvent -private[spark] class SQLListener(conf: SparkConf) extends SparkListener with Logging { +private[sql] class SQLEventRegister extends SparkListenerEventRegister { + + override def getEventClasses(): List[Class[_]] = { + List( + classOf[SparkListenerSQLExecutionStart], + classOf[SparkListenerSQLExecutionEnd]) + } + + override def getListener(): SparkListener = { + new SQLHistoryListener(new SparkConf()) + } + + override def attachUITab(listener: SparkListener, sparkUI: SparkUI): Unit = { + new SQLTab(listener.asInstanceOf[SQLListener], sparkUI) + } +} + +private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { val EXECUTION_ID_KEY = "spark.sql.execution.id" From 927bae84244f900724c3865aac969e4639594760 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 6 Nov 2015 15:17:45 +0800 Subject: [PATCH 14/22] code clean --- .../scala/org/apache/spark/scheduler/SparkListener.scala | 7 ++----- .../org/apache/spark/sql/execution/metric/SQLMetrics.scala | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 336314047463..0bfc008d6ef5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -31,14 +31,11 @@ import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.ui.SparkUI @DeveloperApi -trait SparkListenerEvent { -} +trait SparkListenerEvent @DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) - extends SparkListenerEvent { -} - + extends SparkListenerEvent @DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 0a42824d55c6..8b650c7c081a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -79,7 +79,7 @@ private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetr * A specialized long Accumulable to avoid boxing and unboxing when using Accumulator's * `+=` and `add`. */ -private[sql] class LongSQLMetric(name: String, param: LongSQLMetricParam) +private[sql] class LongSQLMetric private[metric] (name: String, param: LongSQLMetricParam) extends SQLMetric[LongSQLMetricValue, Long](name, param) { override def +=(term: Long): Unit = { From b03d98bcb6854a01b65fd5e43368580dec192482 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Mon, 9 Nov 2015 13:24:04 +0800 Subject: [PATCH 15/22] minor fix --- .../main/scala/org/apache/spark/sql/SQLContext.scala | 11 ++++++++--- .../spark/sql/execution/metric/SQLMetrics.scala | 2 +- .../apache/spark/sql/execution/ui/SQLListener.scala | 5 ++--- .../org/apache/spark/sql/test/SharedSQLContext.scala | 1 + 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 036fb15d9903..90b0e98c6fa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1285,6 +1285,10 @@ object SQLContext { Option(instantiatedContext.get()) } + private[sql] def clearSqlListener(listener: SQLListener): Unit = { + sqlListener.compareAndSet(listener, 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 @@ -1335,9 +1339,10 @@ object SQLContext { private[sql] def createListenerAndUI(sc: SparkContext): SQLListener = { if (sqlListener.get() == null) { val listener = new SQLListener(sc.conf) - sqlListener.compareAndSet(null, listener) - sc.addSparkListener(listener) - sc.ui.foreach(new SQLTab(listener, _)) + if (sqlListener.compareAndSet(null, listener)) { + sc.addSparkListener(listener) + sc.ui.foreach(new SQLTab(listener, _)) + } } sqlListener.get() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 8b650c7c081a..d2108b93c381 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -79,7 +79,7 @@ private[sql] class LongSQLMetricValue(private var _value : Long) extends SQLMetr * A specialized long Accumulable to avoid boxing and unboxing when using Accumulator's * `+=` and `add`. */ -private[sql] class LongSQLMetric private[metric] (name: String, param: LongSQLMetricParam) +private[sql] class LongSQLMetric private[metric](name: String, param: LongSQLMetricParam) extends SQLMetric[LongSQLMetricValue, Long](name, param) { override def +=(term: Long): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 1f63b51f3f4a..96b227770cc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetricValue, SQLMetricParam} import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} @@ -59,8 +60,6 @@ private[sql] class SQLEventRegister extends SparkListenerEventRegister { private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { - val EXECUTION_ID_KEY = "spark.sql.execution.id" - private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) private val activeExecutions = mutable.HashMap[Long, SQLExecutionUIData]() @@ -112,7 +111,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } override def onJobStart(jobStart: SparkListenerJobStart): Unit = { - val executionIdString = jobStart.properties.getProperty(EXECUTION_ID_KEY) + val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) if (executionIdString == null) { // This is not a job created by SQL return diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 963d10eed62e..628ebe0700fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -56,6 +56,7 @@ trait SharedSQLContext extends SQLTestUtils { try { if (_ctx != null) { _ctx.sparkContext.stop() + SQLContext.clearSqlListener(_ctx.listener) _ctx = null } } finally { From 51f913bed91dd95029b404092e8cd7c25b02cad6 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Tue, 10 Nov 2015 10:04:09 +0800 Subject: [PATCH 16/22] Fix unit tests --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- .../scala/org/apache/spark/sql/test/SharedSQLContext.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 90b0e98c6fa6..6e6a3f2302b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1285,8 +1285,8 @@ object SQLContext { Option(instantiatedContext.get()) } - private[sql] def clearSqlListener(listener: SQLListener): Unit = { - sqlListener.compareAndSet(listener, null) + private[sql] def clearSqlListener(): Unit = { + sqlListener.set(null) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 628ebe0700fc..e7b376548787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -42,6 +42,7 @@ trait SharedSQLContext extends SQLTestUtils { * Initialize the [[TestSQLContext]]. */ protected override def beforeAll(): Unit = { + SQLContext.clearSqlListener() if (_ctx == null) { _ctx = new TestSQLContext } @@ -56,7 +57,6 @@ trait SharedSQLContext extends SQLTestUtils { try { if (_ctx != null) { _ctx.sparkContext.stop() - SQLContext.clearSqlListener(_ctx.listener) _ctx = null } } finally { From bca3f5ffca9a9e6f5c56633812432892af66b4e7 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 12 Nov 2015 14:07:36 +0800 Subject: [PATCH 17/22] Address comments --- .../spark/scheduler/SparkListener.scala | 22 ++--- .../scala/org/apache/spark/ui/SparkUI.scala | 16 ++- .../org/apache/spark/util/JsonProtocol.scala | 32 ++---- ...spark.scheduler.SparkListenerEventRegister | 1 - ...ache.spark.scheduler.SparkListenerRegister | 1 + .../spark/sql/execution/SparkPlanInfo.scala | 3 +- .../spark/sql/execution/ui/SQLListener.scala | 98 ++++++++++--------- 7 files changed, 81 insertions(+), 92 deletions(-) delete mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 0bfc008d6ef5..86ceaa63808f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,10 +19,12 @@ package org.apache.spark.scheduler import java.util.Properties +import com.fasterxml.jackson.annotation.JsonTypeInfo + import scala.collection.Map import scala.collection.mutable -import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.{Logging, SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -31,6 +33,7 @@ import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.ui.SparkUI @DeveloperApi +@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "Event") trait SparkListenerEvent @DeveloperApi @@ -132,23 +135,14 @@ case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent /** - * Interface for registering events defined in other modules like SQL. + * Interface for registering listeners defined in other modules like SQL, which are used to + * rebuild the history UI. */ -trait SparkListenerEventRegister { - /** - * Get the class of events to register. - */ - def getEventClasses(): List[Class[_]] - +trait SparkListenerRegister { /** * Get a listener used when rebuilding the history UI. */ - def getListener(): SparkListener - - /** - * Attach a UI Tab to the spark UI if necessary. - */ - def attachUITab(listener: SparkListener, sparkUI: SparkUI) + def getListener(conf: SparkConf, sparkUI: SparkUI): SparkListener } /** diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 30780118ef7b..5f22389841a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,11 +17,13 @@ package org.apache.spark.ui -import java.util.Date +import java.util.{Date, ServiceLoader} + +import scala.collection.JavaConverters._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, UIRoot} -import org.apache.spark.util.JsonProtocol +import org.apache.spark.util.Utils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -128,6 +130,11 @@ private[spark] object SparkUI { val DEFAULT_RETAINED_STAGES = 1000 val DEFAULT_RETAINED_JOBS = 1000 + val listenerRegisters: Iterable[SparkListenerRegister] = { + val loader = Utils.getContextOrSparkClassLoader + ServiceLoader.load(classOf[SparkListenerRegister], loader).asScala + } + def getUIPort(conf: SparkConf): Int = { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) } @@ -153,10 +160,9 @@ private[spark] object SparkUI { startTime: Long): SparkUI = { val sparkUI = create( None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) - JsonProtocol.eventRegisters.foreach { eventRegister => - val listener = eventRegister.getListener() + listenerRegisters.foreach { listenRegister => + val listener = listenRegister.getListener(conf, sparkUI) listenerBus.addListener(listener) - eventRegister.attachUITab(listener, sparkUI) } sparkUI } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 52f6e0c67c27..ee972aea8f7f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -17,24 +17,24 @@ package org.apache.spark.util -import java.util.{ServiceLoader, Properties, UUID} +import java.util.{Properties, UUID} -import org.apache.spark.scheduler.cluster.ExecutorInfo +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule import scala.collection.JavaConverters._ import scala.collection.Map -import scala.collection.mutable.ListBuffer -import org.json4s.{ShortTypeHints, DefaultFormats} +import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ -import org.json4s.jackson.Serialization.{read, write} import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ /** @@ -55,22 +55,9 @@ import org.apache.spark.storage._ private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. - // Events defined in other modules are allowed to be registered through implementing the - // SparkListenerEventRegister trait. These events are written to Json using Jackson. - val eventRegisters: Iterable[SparkListenerEventRegister] = { - val loader = Utils.getContextOrSparkClassLoader - ServiceLoader.load(classOf[SparkListenerEventRegister], loader).asScala - } - - var eventClasses = new ListBuffer[Class[_]]() - eventRegisters.foreach { eventRegister => - eventClasses ++= eventRegister.getEventClasses() - } + private implicit val format = DefaultFormats - private implicit val format = new DefaultFormats { - override val typeHintFieldName = "Event" - override val typeHints = ShortTypeHints(eventClasses.toList) - } + val mapper = new ObjectMapper().registerModule(DefaultScalaModule) /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | @@ -114,7 +101,7 @@ private[spark] object JsonProtocol { executorMetricsUpdateToJson(metricsUpdate) case blockUpdated: SparkListenerBlockUpdated => throw new MatchError(blockUpdated) // TODO(ekl) implement this - case _ => parse(write(event)) + case _ => parse(mapper.writeValueAsString(event)) } } @@ -523,7 +510,8 @@ private[spark] object JsonProtocol { case `executorRemoved` => executorRemovedFromJson(json) case `logStart` => logStartFromJson(json) case `metricsUpdate` => executorMetricsUpdateFromJson(json) - case other => read[SparkListenerEvent](compact(render(json))) + case other => mapper.readValue(compact(render(json)), Utils.classForName(other)) + .asInstanceOf[SparkListenerEvent] } } diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister deleted file mode 100644 index e9e60b6260e0..000000000000 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerEventRegister +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.sql.execution.ui.SQLEventRegister diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister new file mode 100644 index 000000000000..6e884a057988 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister @@ -0,0 +1 @@ +org.apache.spark.sql.execution.ui.SQLListenerRegister diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index d8d65e51a704..51fbc68d9b1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -31,8 +31,7 @@ class SparkPlanInfo( val simpleString: String, val children: Seq[SparkPlanInfo], val metrics: Seq[SQLMetricInfo] -) { -} +) private[sql] object SparkPlanInfo { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 96b227770cc2..ac127041f15e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -41,20 +41,10 @@ case class SparkListenerSQLExecutionStart( case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) extends SparkListenerEvent -private[sql] class SQLEventRegister extends SparkListenerEventRegister { +private[sql] class SQLListenerRegister extends SparkListenerRegister { - override def getEventClasses(): List[Class[_]] = { - List( - classOf[SparkListenerSQLExecutionStart], - classOf[SparkListenerSQLExecutionEnd]) - } - - override def getListener(): SparkListener = { - new SQLHistoryListener(new SparkConf()) - } - - override def attachUITab(listener: SparkListener, sparkUI: SparkUI): Unit = { - new SQLTab(listener.asInstanceOf[SQLListener], sparkUI) + override def getListener(conf: SparkConf, sparkUI: SparkUI): SparkListener = { + new SQLHistoryListener(conf, sparkUI) } } @@ -224,42 +214,39 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } } - override def onOtherEvent(event: SparkListenerEvent): Unit = { - event match { - case executionStart: SparkListenerSQLExecutionStart => - val physicalPlanGraph = SparkPlanGraph(executionStart.sparkPlanInfo) - val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) - } - val executionUIData = new SQLExecutionUIData( - executionStart.executionId, - executionStart.description, - executionStart.details, - executionStart.physicalPlanDescription, - physicalPlanGraph, - sqlPlanMetrics.toMap, - executionStart.time) - synchronized { - activeExecutions(executionStart.executionId) = executionUIData - _executionIdToData(executionStart.executionId) = executionUIData - } - case executionEnd: SparkListenerSQLExecutionEnd => - synchronized { - _executionIdToData.get(executionEnd.executionId).foreach { executionUIData => - executionUIData.completionTime = Some(executionEnd.time) - if (!executionUIData.hasRunningJobs) { - // onExecutionEnd happens after all "onJobEnd"s - // So we should update the execution lists. - markExecutionFinished(executionEnd.executionId) - } else { - // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. - // Then we don't if the execution is successful, so let the last onJobEnd updates the - // execution lists. - } - } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case executionStart: SparkListenerSQLExecutionStart => + val physicalPlanGraph = SparkPlanGraph(executionStart.sparkPlanInfo) + val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => + node.metrics.map(metric => metric.accumulatorId -> metric) + } + val executionUIData = new SQLExecutionUIData( + executionStart.executionId, + executionStart.description, + executionStart.details, + executionStart.physicalPlanDescription, + physicalPlanGraph, + sqlPlanMetrics.toMap, + executionStart.time) + synchronized { + activeExecutions(executionStart.executionId) = executionUIData + _executionIdToData(executionStart.executionId) = executionUIData + } + case executionEnd: SparkListenerSQLExecutionEnd => synchronized { + _executionIdToData.get(executionEnd.executionId).foreach { executionUIData => + executionUIData.completionTime = Some(executionEnd.time) + if (!executionUIData.hasRunningJobs) { + // onExecutionEnd happens after all "onJobEnd"s + // So we should update the execution lists. + markExecutionFinished(executionEnd.executionId) + } else { + // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. + // Then we don't if the execution is successful, so let the last onJobEnd updates the + // execution lists. } - case _ => // Ignore + } } + case _ => // Ignore } private def markExecutionFinished(executionId: Long): Unit = { @@ -324,11 +311,16 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } -private[spark] class SQLHistoryListener(conf: SparkConf) extends SQLListener(conf) { +private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) + extends SQLListener(conf) { + + var sqlTabAttached = false + override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { // Do nothing } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { updateTaskAccumulatorValues( taskEnd.taskInfo.taskId, @@ -339,6 +331,16 @@ private[spark] class SQLHistoryListener(conf: SparkConf) extends SQLListener(con }.toMap, finishTask = true) } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case executionStart: SparkListenerSQLExecutionStart => + if (!sqlTabAttached) { + new SQLTab(this, sparkUI) + sqlTabAttached = true + } + super.onOtherEvent(event) + case _ => super.onOtherEvent(event) + } } /** From 60033f8da8bd564c01811c25dc9dbfdf897896b7 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 12 Nov 2015 14:28:31 +0800 Subject: [PATCH 18/22] Fix RAT test --- .rat-excludes | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.rat-excludes b/.rat-excludes index 02f7ec606cf8..71b60bcdab9b 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -82,5 +82,5 @@ INDEX gen-java.* .*avpr org.apache.spark.sql.sources.DataSourceRegister -org.apache.spark.scheduler.SparkListenerEventRegister +org.apache.spark.scheduler.SparkListenerRegister .*parquet From fe5c16529d821098658cb61ca0f0a21d2a568270 Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 12 Nov 2015 14:36:20 +0800 Subject: [PATCH 19/22] Fix style --- .../scala/org/apache/spark/sql/execution/SparkPlanInfo.scala | 3 +-- .../org/apache/spark/sql/execution/metric/SQLMetricInfo.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 51fbc68d9b1f..486ce34064e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -30,8 +30,7 @@ class SparkPlanInfo( val nodeName: String, val simpleString: String, val children: Seq[SparkPlanInfo], - val metrics: Seq[SQLMetricInfo] -) + val metrics: Seq[SQLMetricInfo]) private[sql] object SparkPlanInfo { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala index 6f1c9f6a3d39..2708219ad348 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetricInfo.scala @@ -27,5 +27,4 @@ import org.apache.spark.annotation.DeveloperApi class SQLMetricInfo( val name: String, val accumulatorId: Long, - val metricParam: String) { -} + val metricParam: String) From 8d94707d4e44741d5e3c24a146d2bf526cf884ce Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 19 Nov 2015 11:24:50 +0800 Subject: [PATCH 20/22] Address vanzin's comments --- .rat-excludes | 2 +- .../org/apache/spark/scheduler/SparkListener.scala | 12 ++++++------ .../main/scala/org/apache/spark/ui/SparkUI.scala | 14 ++++++-------- .../scala/org/apache/spark/util/JsonProtocol.scala | 8 ++++---- ...che.spark.scheduler.SparkHistoryListenerFactory | 1 + ...rg.apache.spark.scheduler.SparkListenerRegister | 1 - .../spark/sql/execution/metric/SQLMetrics.scala | 4 ++-- .../spark/sql/execution/ui/SQLListener.scala | 10 +++++----- 8 files changed, 25 insertions(+), 27 deletions(-) create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkHistoryListenerFactory delete mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister diff --git a/.rat-excludes b/.rat-excludes index 71b60bcdab9b..7262c960ed6b 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -82,5 +82,5 @@ INDEX gen-java.* .*avpr org.apache.spark.sql.sources.DataSourceRegister -org.apache.spark.scheduler.SparkListenerRegister +org.apache.spark.scheduler.SparkHistoryListenerFactory .*parquet diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86ceaa63808f..075a7f13172d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,11 +19,11 @@ package org.apache.spark.scheduler import java.util.Properties -import com.fasterxml.jackson.annotation.JsonTypeInfo - import scala.collection.Map import scala.collection.mutable +import com.fasterxml.jackson.annotation.JsonTypeInfo + import org.apache.spark.{Logging, SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics @@ -135,14 +135,14 @@ case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent /** - * Interface for registering listeners defined in other modules like SQL, which are used to + * Interface for creating history listeners defined in other modules like SQL, which are used to * rebuild the history UI. */ -trait SparkListenerRegister { +private[spark] trait SparkHistoryListenerFactory { /** - * Get a listener used when rebuilding the history UI. + * Create listeners used to rebuild the history UI. */ - def getListener(conf: SparkConf, sparkUI: SparkUI): SparkListener + def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] } /** diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 5f22389841a9..c760781303fa 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -130,11 +130,6 @@ private[spark] object SparkUI { val DEFAULT_RETAINED_STAGES = 1000 val DEFAULT_RETAINED_JOBS = 1000 - val listenerRegisters: Iterable[SparkListenerRegister] = { - val loader = Utils.getContextOrSparkClassLoader - ServiceLoader.load(classOf[SparkListenerRegister], loader).asScala - } - def getUIPort(conf: SparkConf): Int = { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) } @@ -160,9 +155,12 @@ private[spark] object SparkUI { startTime: Long): SparkUI = { val sparkUI = create( None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) - listenerRegisters.foreach { listenRegister => - val listener = listenRegister.getListener(conf, sparkUI) - listenerBus.addListener(listener) + + val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory], + Utils.getContextOrSparkClassLoader).asScala + listenerFactories.foreach { listenerFactory => + val listeners = listenerFactory.createListeners(conf, sparkUI) + listeners.foreach(listenerBus.addListener) } sparkUI } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ee972aea8f7f..3f8ed5c3abee 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -19,12 +19,12 @@ package org.apache.spark.util import java.util.{Properties, UUID} -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule - import scala.collection.JavaConverters._ import scala.collection.Map +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ @@ -57,7 +57,7 @@ private[spark] object JsonProtocol { private implicit val format = DefaultFormats - val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkHistoryListenerFactory b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkHistoryListenerFactory new file mode 100644 index 000000000000..507100be9096 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkHistoryListenerFactory @@ -0,0 +1 @@ +org.apache.spark.sql.execution.ui.SQLHistoryListenerFactory diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister deleted file mode 100644 index 6e884a057988..000000000000 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.SparkListenerRegister +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.sql.execution.ui.SQLListenerRegister diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index d2108b93c381..b6d31ab80d82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -154,8 +154,8 @@ private[sql] object SQLMetrics { val longSQLMetricParam = Utils.getFormattedClassName(LongSQLMetricParam) val staticsSQLMetricParam = Utils.getFormattedClassName(StaticsLongSQLMetricParam) val metricParam = metricParamName match { - case `longSQLMetricParam` => LongSQLMetricParam - case `staticsSQLMetricParam` => StaticsLongSQLMetricParam + case `longSQLMetricParam` => LongSQLMetricParam + case `staticsSQLMetricParam` => StaticsLongSQLMetricParam } metricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index ac127041f15e..0516845c869c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -41,10 +41,10 @@ case class SparkListenerSQLExecutionStart( case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) extends SparkListenerEvent -private[sql] class SQLListenerRegister extends SparkListenerRegister { +private[sql] class SQLHistoryListenerFactory extends SparkHistoryListenerFactory { - override def getListener(conf: SparkConf, sparkUI: SparkUI): SparkListener = { - new SQLHistoryListener(conf, sparkUI) + override def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] = { + List(new SQLHistoryListener(conf, sparkUI)) } } @@ -314,7 +314,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) extends SQLListener(conf) { - var sqlTabAttached = false + private var sqlTabAttached = false override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { @@ -333,7 +333,7 @@ private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) } override def onOtherEvent(event: SparkListenerEvent): Unit = event match { - case executionStart: SparkListenerSQLExecutionStart => + case _: SparkListenerSQLExecutionStart => if (!sqlTabAttached) { new SQLTab(this, sparkUI) sqlTabAttached = true From 56f24bafb3f5b9306bf393220d018e0620f1296d Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Thu, 19 Nov 2015 15:37:23 +0800 Subject: [PATCH 21/22] Address hao's comments --- .../sql/execution/metric/SQLMetrics.scala | 6 ++--- .../spark/sql/execution/ui/SQLListener.scala | 27 ++++++++++--------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index b6d31ab80d82..6c0f6f8a52dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -91,7 +91,7 @@ private[sql] class LongSQLMetric private[metric](name: String, param: LongSQLMet } } -private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) +private class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialValue: Long) extends SQLMetricParam[LongSQLMetricValue, Long] { override def addAccumulator(r: LongSQLMetricValue, t: Long): LongSQLMetricValue = r.add(t) @@ -104,9 +104,9 @@ private[sql] class LongSQLMetricParam(val stringValue: Seq[Long] => String, init override def zero: LongSQLMetricValue = new LongSQLMetricValue(initialValue) } -private[sql] object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) +private object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) -private[sql] object StaticsLongSQLMetricParam extends LongSQLMetricParam( +private object StaticsLongSQLMetricParam extends LongSQLMetricParam( (values: Seq[Long]) => { // This is a workaround for SPARK-11013. // We use -1 as initial value of the accumulator, if the accumulator is valid, we will update diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 0516845c869c..e19a1e3e5851 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -215,30 +215,31 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } override def onOtherEvent(event: SparkListenerEvent): Unit = event match { - case executionStart: SparkListenerSQLExecutionStart => - val physicalPlanGraph = SparkPlanGraph(executionStart.sparkPlanInfo) + case SparkListenerSQLExecutionStart(executionId, description, details, + physicalPlanDescription, sparkPlanInfo, time) => + val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => node.metrics.map(metric => metric.accumulatorId -> metric) } val executionUIData = new SQLExecutionUIData( - executionStart.executionId, - executionStart.description, - executionStart.details, - executionStart.physicalPlanDescription, + executionId, + description, + details, + physicalPlanDescription, physicalPlanGraph, sqlPlanMetrics.toMap, - executionStart.time) + time) synchronized { - activeExecutions(executionStart.executionId) = executionUIData - _executionIdToData(executionStart.executionId) = executionUIData + activeExecutions(executionId) = executionUIData + _executionIdToData(executionId) = executionUIData } - case executionEnd: SparkListenerSQLExecutionEnd => synchronized { - _executionIdToData.get(executionEnd.executionId).foreach { executionUIData => - executionUIData.completionTime = Some(executionEnd.time) + case SparkListenerSQLExecutionEnd(executionId, time) => synchronized { + _executionIdToData.get(executionId).foreach { executionUIData => + executionUIData.completionTime = Some(time) if (!executionUIData.hasRunningJobs) { // onExecutionEnd happens after all "onJobEnd"s // So we should update the execution lists. - markExecutionFinished(executionEnd.executionId) + markExecutionFinished(executionId) } else { // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. // Then we don't if the execution is successful, so let the last onJobEnd updates the From 690277e37c76b98ba57bc4bd5b0ae2c87669578d Mon Sep 17 00:00:00 2001 From: Carson Wang Date: Fri, 20 Nov 2015 10:39:40 +0800 Subject: [PATCH 22/22] Remove one empty line --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 3f8ed5c3abee..c4eeb0bf3b50 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -24,7 +24,6 @@ import scala.collection.Map import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule - import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._