Skip to content

Conversation

@carsonwang
Copy link
Contributor

On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:

  1. I added an onOtherEvent method to the SparkListener trait and post all SQL related events to the same event bus.
  2. Two SQL events SparkListenerSQLExecutionStart and SparkListenerSQLExecutionEnd are defined in the sql module.
  3. The new SQL events are written to event log using Jackson.
  4. A new trait SparkHistoryListenerFactory is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using java.util.ServiceLoader.

@SparkQA
Copy link

SparkQA commented Oct 27, 2015

Test build #44419 has finished for PR 9297 at commit d52288b.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This mach really needs `case other:String => throw new IOException(s"Unknown Event $other") message, so that when new events are added in future, the old replay code will at least display a meaningful message. Maybe that's for another pull req though.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added that. Thanks.

@SparkQA
Copy link

SparkQA commented Oct 28, 2015

Test build #44493 has finished for PR 9297 at commit 7a2aced.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 28, 2015

Test build #44499 has finished for PR 9297 at commit caab0ba.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 28, 2015

Test build #44503 has finished for PR 9297 at commit 0af5afe.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Oct 28, 2015

Hi @carsonwang,

This is a really useful addition, but I'm afraid this approach is not really scalable. For example, you have the same problem with the Streaming tab in the UI, and trying to fix that with your approach would just make things worse.

I actually think the underlying code needs some lower level changes before this can be implemented more cleanly. For example, a few things that I thought about in the past:

  • SparkListenerEvent should not be a sealed trait; that way streaming events and sql events can extend it without having to exist in core
  • SparkListener should not be the root of the listener hierarchy; SparkFirehoseListener should, and SparkListener should be a specialization of that trait that provides event type-specific methods.
  • events should be written to the logs using a library like Jackson instead of having to modify JsonProtocol for each new event. The current approach is just not scalable. The main issue here is backwards compatibility, but I think this is a much needed change in this part of the code.

With those in place, the only thing left is a protocol so that the history server can feed events to backend-specific listeners; that could be done by having the history server expose a new trait that is loaded at runtime using something like java.util.ServiceLoader; then there could be an implementation of that trait in sql (and another one in streaming) to process events read from the logs and add stuff to the UI.

What do you think? I really would like to avoid moving more stuff to the core; it feels like this would be a pretty good opportunity to work on some of the outstanding issues with this whole part of the code.

@carsonwang
Copy link
Contributor Author

@vanzin Thanks a lot for the comment. This sounds great and is very helpful. I agree it is not a good idea to move more stuff to the core. The underlying code change you mentioned sounds a lot of works to do, especially when we consider backwards compatibility. Can we apply part of them so that we can support SQL UI on the history server in this PR and also avoid moving the SQL classes to the core? I was thinking adding a toJson method to the SparkListenerEvent. As you mentioned when it is not a sealed trait, we can define sql events in the sql module. For existing events in the core, we still use the JsonProtocol in the toJson method to convert the event to json. For sql events, we can define toJson method in the sql module. This way we will be able to write the sql events to the log. Then we can use something you mentioned to feed events from the history server to the sql listener.

I know this might still not be very scalable. But we don't make things worse once we are able to avoid moving sql classes to the core and also support SQL UI on the history server. Do you think this is doable for this ticket?

@vanzin
Copy link
Contributor

vanzin commented Nov 1, 2015

I was thinking adding a toJson method to the SparkListenerEvent.

I'm not a big fan of allowing events to write custom serialization code. I think a better approach would be to change JsonProtocol to use Jackson when the event doesn't match any of the currently known event types. Then later we could add a flag that makes JsonProtocol use Jackson for all event types.

Mostly that's for backwards compatibility, so that event that are currently written to the event log don't change format. We can then document that the old format is deprecated and will be switched to the new, Jackson-based one in a future release.

@carsonwang
Copy link
Contributor Author

@vanzin , your suggestions are very helpful. I have updated the code following the ideas. I use Jackson to serialize the SQL events and also introduce a trait to register the events and feed them to the SQL listener. The SQL implementation of the trait is loaded at runtime using java.util.ServiceLoader. Can you please help take a look?

@SparkQA
Copy link

SparkQA commented Nov 6, 2015

Test build #45201 has finished for PR 9297 at commit 1954d71.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * trait SparkListenerEvent\n * trait SparkListenerEventRegister\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@SparkQA
Copy link

SparkQA commented Nov 6, 2015

Test build #45203 has finished for PR 9297 at commit 927bae8.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * trait SparkListenerEventRegister\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@SparkQA
Copy link

SparkQA commented Nov 9, 2015

Test build #45346 has finished for PR 9297 at commit b03d98b.

  • This patch fails from timeout after a configured wait of 250m.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * trait SparkListenerEventRegister\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@SparkQA
Copy link

SparkQA commented Nov 10, 2015

Test build #45470 has finished for PR 9297 at commit 51f913b.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 19, 2015

Test build #46320 has finished for PR 9297 at commit 56f24ba.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

super nit: get rid of this empty line.

@vanzin
Copy link
Contributor

vanzin commented Nov 19, 2015

LGTM. retest this please

@SparkQA
Copy link

SparkQA commented Nov 20, 2015

Test build #46373 has finished for PR 9297 at commit 56f24ba.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@carsonwang
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Nov 20, 2015

Test build #46386 has finished for PR 9297 at commit 690277e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@SparkQA
Copy link

SparkQA commented Nov 20, 2015

Test build #46387 has finished for PR 9297 at commit 690277e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@vanzin
Copy link
Contributor

vanzin commented Nov 24, 2015

retest this please

@SparkQA
Copy link

SparkQA commented Nov 25, 2015

Test build #46637 has finished for PR 9297 at commit 690277e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):\n * class SparkPlanInfo(\n * class SQLMetricInfo(\n * case class SparkListenerSQLExecutionStart(\n * case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)\n

@steveloughran
Copy link
Contributor

While I support this feature, people already see problems with scanning/loading app histories off the filesystem. This has the potential to increase it significantly. I think it helps justify having those fs logs save some metadata summary (started, ended, spark version, ...) alongside the logs, so at least the app listing operation can bypass the scan. Separate JIRA, obviously

@vanzin
Copy link
Contributor

vanzin commented Nov 25, 2015

@steveloughran while that's true, this change shouldn't make things that much worse; things are already pretty bad for applications that generate large logs.

I'm merging this to master.

@asfgit asfgit closed this in cc243a0 Nov 25, 2015
@JoshRosen
Copy link
Contributor

I just observed a SQLListenerMemoryLeakSuite memory leak test failure in Jenkins. Do you think that it could somehow be related to this patch?

Here's the failing test:

https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-pre-YARN/HADOOP_VERSION=2.0.0-mr1-cdh4.1.2,label=spark-test/4896/

No need to revert or take drastic action yet, but let's keep an eye on that suite to see if the test fails in any more builds; you can monitor this test at https://spark-tests.appspot.com/tests/org.apache.spark.sql.execution.ui.SQLListenerMemoryLeakSuite/no%20memory%20leak, an experimental dashboard that I've been building to help investigate these types of flaky test issues.

@vanzin
Copy link
Contributor

vanzin commented Nov 26, 2015

I can't really say, I'm not too familiar with the SQL backend here. @chenghao-intel took a look, maybe he has some insight.

@chenghao-intel
Copy link
Contributor

Seems failed again, we are debugging on it now, will keep you posted if anything find.

@carsonwang
Copy link
Contributor Author

I just submitted #9991 to fix the test failure. Details are described in the new PR. Thanks all!

@JoshRosen
Copy link
Contributor

Just curious: are there any end-to-end tests for this feature? Would it be very hard to add some?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given that it's still legal to have multiple active SQLContexts in an application (although this is not a good practice), isn't this a user-facing behavioral change? If a user creates multiple SQLContexts, then they will now only have one SQL tab in the UI and one SQLListener, which is different than the old behavior.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, is the only purpose of sqlListener to prevent multiple listeners from being created at the same time? If so, I think it would be better to use an AtomicBoolean so that we don't create another strong reference to a SQLListener, which might have a lot of internal state that could lead to memory leaks.

@vanzin
Copy link
Contributor

vanzin commented Nov 30, 2015

I don't think so. Tests would have to be added to one of the sql/ modules since you can't really test those events from the core/ module...

In core/ the only thing that can be tested is that Jackson is doing the right thing when unknown event types are written or present in the event log.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this method is going to be a no-op under certain circumstances then those circumstances should be documented here.

@JoshRosen
Copy link
Contributor

@vanzin, in that case, why can't those tests live in the SQL module? I think that we should be able to add a test where we create a SparkContext + SQLContext with event logging enabled, run some queries, stop the context, then programmatically instantiate a HistoryServer, reconstitute the UI from the event logs, and perform some assertions over the loaded UI (e.g. check the number of SQL tabs, check that clicking on the SQL tab returns a page with some expected strings, etc). We could easily define some test helper classes to make this easy to write.

@vanzin
Copy link
Contributor

vanzin commented Nov 30, 2015

I'm not saying they can't live there, I'm just saying they have to be written (since they don't exist) and when they are, they'll have to live there.

@JoshRosen
Copy link
Contributor

Because this PR broke the Master Maven tests, #9991 isn't read to merge, and there's not an easy way to ignore failing tests (since the problem is broader in scope than that single test), I'm going to revert this for now. Let's aim to resubmit a new PR which addresses the concerns being discussed in #9991.

asfgit pushed a commit that referenced this pull request Nov 30, 2015
This reverts commit cc243a0 / PR #9297

I'm reverting this because it broke SQLListenerMemoryLeakSuite in the master Maven builds.

See #9991 for a discussion of why this broke the tests.
ghost pushed a commit to dbtsai/spark that referenced this pull request Dec 4, 2015
Resubmit apache#9297 and apache#9991
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an onOtherEvent method to the SparkListener trait and post all SQL related events to the same event bus.
2. Two SQL events SparkListenerSQLExecutionStart and SparkListenerSQLExecutionEnd are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4. A new trait SparkHistoryListenerFactory is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using java.util.ServiceLoader.

Author: Carson Wang <[email protected]>

Closes apache#10061 from carsonwang/SqlHistoryUI.
Parth-Brahmbhatt pushed a commit to Parth-Brahmbhatt/spark that referenced this pull request Jul 25, 2016
Resubmit apache#9297 and apache#9991
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an onOtherEvent method to the SparkListener trait and post all SQL related events to the same event bus.
2. Two SQL events SparkListenerSQLExecutionStart and SparkListenerSQLExecutionEnd are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4. A new trait SparkHistoryListenerFactory is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using java.util.ServiceLoader.

Author: Carson Wang <[email protected]>

Closes apache#10061 from carsonwang/SqlHistoryUI.

# Conflicts:
#	sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala
#	sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala

Added more features:
# More details to the SparkPlanInfo
# Added the execution plan to action

# Conflicts:
#	sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala
#	sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants