Skip to content

[SPARK-18143][SQL]Ignore Structured Streaming event logs to avoid breaking history server #15663

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 5 commits into from

Conversation

zsxwing
Copy link
Member

@zsxwing zsxwing commented Oct 27, 2016

What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.

How was this patch tested?

  • Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as structured-streaming-query-event-logs-2.0.0.txt and structured-streaming-query-event-logs-2.0.1.txt
  • The new added test makes sure ReplayListenerBus will skip these bad jsons.

@zsxwing
Copy link
Member Author

zsxwing commented Oct 27, 2016

cc @marmbrus @tdas

@zsxwing
Copy link
Member Author

zsxwing commented Oct 27, 2016

@rxin do we need to cancel 2.0.2-RC1 because of this issue?

case e: java.lang.ClassNotFoundException if
e.getMessage == "org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress" ||
e.getMessage == "org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated"
=>
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: formatting is weird.
wont it be easier if you defined a set with these class names, and then use "contains"?

@tdas
Copy link
Contributor

tdas commented Oct 27, 2016

one comment, but overall looks good.

@SparkQA
Copy link

SparkQA commented Oct 27, 2016

Test build #67664 has finished for PR 15663 at commit 5feb1bd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 27, 2016

Test build #67666 has finished for PR 15663 at commit 3aedd53.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 28, 2016

Test build #67678 has finished for PR 15663 at commit 4dd84eb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 28, 2016

Test build #67680 has finished for PR 15663 at commit 0f51b4e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -206,6 +209,45 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
assert(queryQueryTerminated.exception === newQueryTerminated.exception)
}

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 test is in sql/core, why are the resources in history server?

Copy link
Contributor

Choose a reason for hiding this comment

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

I should have noticed this earlier. Sorry!

Copy link
Contributor

Choose a reason for hiding this comment

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

Duh! its in sql/core :(

But good to create a structured-streaming directory rather than history-server

Copy link
Member Author

Choose a reason for hiding this comment

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

Renamed

@zsxwing
Copy link
Member Author

zsxwing commented Oct 29, 2016

Updated. 2.0.1 event logs don't include the "stackTrace" field.

@SparkQA
Copy link

SparkQA commented Oct 29, 2016

Test build #67731 has finished for PR 15663 at commit 729d50d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 29, 2016

Test build #67734 has finished for PR 15663 at commit d30a3a5.

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

@zsxwing
Copy link
Member Author

zsxwing commented Oct 29, 2016

retest this please

1 similar comment
@zsxwing
Copy link
Member Author

zsxwing commented Oct 29, 2016

retest this please

@SparkQA
Copy link

SparkQA commented Oct 29, 2016

Test build #67742 has finished for PR 15663 at commit d30a3a5.

  • This patch fails from timeout after a configured wait of 250m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 29, 2016

Test build #67751 has finished for PR 15663 at commit d30a3a5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rxin
Copy link
Contributor

rxin commented Oct 31, 2016

cc @tdas / @marmbrus

does this change lgtu?

@tdas
Copy link
Contributor

tdas commented Oct 31, 2016

LGTM. Merging to master and 2.0

@asfgit asfgit closed this in d2923f1 Oct 31, 2016
@tdas
Copy link
Contributor

tdas commented Oct 31, 2016

Okay, I merged only to master, this does not merge cleanly to 2.0. I am on vacation so i cant test out the 2.0 fix quickly, better make a separate PR for 2.0

asfgit pushed a commit that referenced this pull request Oct 31, 2016
…eaking history server (branch 2.0)

## What changes were proposed in this pull request?

Backport #15663 to branch-2.0 and fixed conflicts in `ReplayListenerBus`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15695 from zsxwing/fix-event-log-2.0.
robert3005 pushed a commit to palantir/spark that referenced this pull request Nov 1, 2016
…eaking history server

## What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes apache#15663 from zsxwing/fix-event-log.
@zsxwing zsxwing deleted the fix-event-log branch November 17, 2016 22:31
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…eaking history server

## What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes apache#15663 from zsxwing/fix-event-log.
asfgit pushed a commit that referenced this pull request Dec 13, 2017
…file

## What changes were proposed in this pull request?

While spark code changes, there are new events in event log: #19649
And we used to maintain a whitelist to avoid exceptions: #15663
Currently Spark history server will stop parsing on unknown events or unrecognized properties. We may still see part of the UI data.
For better compatibility, we can ignore unknown events and parse through the log file.

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19953 from gengliangwang/ReplayListenerBus.
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.

4 participants