-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[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
Conversation
@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" | ||
=> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: formatting is weird.
wont it be easier if you defined a set with these class names, and then use "contains"?
one comment, but overall looks good. |
Test build #67664 has finished for PR 15663 at commit
|
Test build #67666 has finished for PR 15663 at commit
|
Test build #67678 has finished for PR 15663 at commit
|
Test build #67680 has finished for PR 15663 at commit
|
@@ -206,6 +209,45 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { | |||
assert(queryQueryTerminated.exception === newQueryTerminated.exception) | |||
} | |||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this test is in sql/core, why are the resources in history server?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I should have noticed this earlier. Sorry!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Duh! its in sql/core :(
But good to create a structured-streaming directory rather than history-server
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Renamed
Updated. 2.0.1 event logs don't include the |
Test build #67731 has finished for PR 15663 at commit
|
Test build #67734 has finished for PR 15663 at commit
|
retest this please |
1 similar comment
retest this please |
Test build #67742 has finished for PR 15663 at commit
|
Test build #67751 has finished for PR 15663 at commit
|
LGTM. Merging to master and 2.0 |
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 |
…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.
…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.
…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.
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?
structured-streaming-query-event-logs-2.0.0.txt
andstructured-streaming-query-event-logs-2.0.1.txt