Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit c4979f6

Browse files
committed
[SPARK-18655][SS] Ignore Structured Streaming 2.0.2 logs in history server
## What changes were proposed in this pull request? As `queryStatus` in StreamingQueryListener events was removed in apache#15954, parsing 2.0.2 structured streaming logs will throw the following errror: ``` [info] com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException: Unrecognized field "queryStatus" (class org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent), not marked as ignorable (2 known properties: "id", "exception"]) [info] at [Source: {"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null}; line: 1, column: 521] (through reference chain: org.apache.spark.sql.streaming.QueryTerminatedEvent["queryStatus"]) [info] at com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException.from(UnrecognizedPropertyException.java:51) [info] at com.fasterxml.jackson.databind.DeserializationContext.reportUnknownProperty(DeserializationContext.java:839) [info] at com.fasterxml.jackson.databind.deser.std.StdDeserializer.handleUnknownProperty(StdDeserializer.java:1045) [info] at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperty(BeanDeserializerBase.java:1352) [info] at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperties(BeanDeserializerBase.java:1306) [info] at com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:453) [info] at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1099) ... ``` This PR just ignores such errors and adds a test to make sure we can read 2.0.2 logs. ## How was this patch tested? `query-event-logs-version-2.0.2.txt` has all types of events generated by Structured Streaming in Spark 2.0.2. `testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2")` verified we can load them without any error. Author: Shixiong Zhu <[email protected]> Closes apache#16085 from zsxwing/SPARK-18655.
1 parent 93e9d88 commit c4979f6

File tree

3 files changed

+20
-0
lines changed

3 files changed

+20
-0
lines changed

core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.io.{InputStream, IOException}
2222
import scala.io.Source
2323

2424
import com.fasterxml.jackson.core.JsonParseException
25+
import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException
2526
import org.json4s.jackson.JsonMethods._
2627

2728
import org.apache.spark.internal.Logging
@@ -87,6 +88,12 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging {
8788
// Ignore events generated by Structured Streaming in Spark 2.0.0 and 2.0.1.
8889
// It's safe since no place uses them.
8990
logWarning(s"Dropped incompatible Structured Streaming log: $currentLine")
91+
case e: UnrecognizedPropertyException if e.getMessage != null && e.getMessage.startsWith(
92+
"Unrecognized field \"queryStatus\" " +
93+
"(class org.apache.spark.sql.streaming.StreamingQueryListener$") =>
94+
// Ignore events generated by Structured Streaming in Spark 2.0.2
95+
// It's safe since no place uses them.
96+
logWarning(s"Dropped incompatible Structured Streaming log: $currentLine")
9097
case jpe: JsonParseException =>
9198
// We can only ignore exception from last line of the file that might be truncated
9299
// the last entry may not be the very last line in the event log, but we treat it
Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryStartedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491481350,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"-","inputRate":0.0,"processingRate":0.0,"triggerDetails":{}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[-]"},"triggerDetails":{}}}
2+
{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491493386,"inputRate":83.33333333333333,"processingRate":0.5773672055427251,"latency":1738.0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":83.33333333333333,"processingRate":0.5773672055427251,"triggerDetails":{"latency.getBatch.source":"39","numRows.input.source":"1","latency.getOffset.source":"91","triggerId":"0"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{"timestamp.afterGetBatch":"1480491491817","latency.offsetLogWrite":"26","timestamp.triggerStart":"1480491491653","triggerId":"0","timestamp.triggerFinish":"1480491493385","latency.fullTrigger":"1732","latency.getBatch.total":"44","timestamp.afterGetOffset":"1480491491772","numRows.input.total":"1","isTriggerActive":"false","latency.optimizer":"406","latency.getOffset.total":"91","isDataPresentInTrigger":"true"}}}
3+
{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null}
4+
{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-0","id":0,"timestamp":1480491812530,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getBatch.source":"25","latency.getOffset.source":"65","triggerId":"0"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[-]"},"triggerDetails":{}},"exception":"org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): org.apache.spark.SparkException: Task failed while writing rows.\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:183)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:155)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:153)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:172)\n\t... 8 more\n\nDriver stacktrace:\n\tat org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)\n\tat scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)\n\tat scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat scala.Option.foreach(Option.scala:257)\n\tat org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1667)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1622)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1611)\n\tat org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1873)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1886)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1906)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.write(FileStreamSink.scala:151)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSink.addBatch(FileStreamSink.scala:70)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:437)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply$mcZ$sp(StreamExecution.scala:225)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:213)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:213)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$reportTimeTaken(StreamExecution.scala:656)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:212)\n\tat org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:208)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:142)\nCaused by: org.apache.spark.SparkException: Task failed while writing rows.\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:183)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:155)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:153)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:172)\n\t... 8 more\n"}
5+
{"Event":"SparkListenerApplicationEnd","Timestamp":1480491541552}

sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -207,6 +207,14 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
207207
testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.1.txt")
208208
}
209209

210+
testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2") {
211+
// query-event-logs-version-2.0.2.txt has all types of events generated by
212+
// Structured Streaming in Spark 2.0.2.
213+
// SparkListenerApplicationEnd is the only valid event and it's the last event. We use it
214+
// to verify that we can skip broken jsons generated by Structured Streaming.
215+
testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.2.txt")
216+
}
217+
210218
private def testReplayListenerBusWithBorkenEventJsons(fileName: String): Unit = {
211219
val input = getClass.getResourceAsStream(s"/structured-streaming/$fileName")
212220
val events = mutable.ArrayBuffer[SparkListenerEvent]()

0 commit comments

Comments
 (0)