Ahmed Hussein created SPARK-43340:
-------------------------------------
Summary: JsonProtocol is not backward compatible
Key: SPARK-43340
URL: https://issues.apache.org/jira/browse/SPARK-43340
Project: Spark
Issue Type: Bug
Components: Spark Core
Affects Versions: 3.4.0, 3.5.0
Reporter: Ahmed Hussein
Fix For: 3.4.1, 3.5.0
Recently I was testing with some 3.0.2 eventlogs.
The SHS-3.4+ does not interpret failed jobs/ failed SQLs correctly.
Instead it will list them as "Incomplete/Active" whereas it should be listed as
"Failed".
The problem is due to missing fields in eventlogs generated by previous
versions. In this case the eventlog does not have "Stack Trace" field which
causes a NPE
```
{"Event":"SparkListenerJobEnd","Job ID":31,"Completion Time":1616171909785,"Job
Result":\{"Result":"JobFailed","Exception":{"Message":"Job aborted"}}}
```
The SHS output
```
23/05/01 21:57:16 INFO FsHistoryProvider: Parsing
file:/Users/ahussein/workspace/repos/spark-rapids-tools/issues/epic-108/eventlogs/spark-340/nds_q86_fail_test
to re-build UI...
23/05/01 21:57:17 ERROR ReplayListenerBus: Exception parsing Spark event log:
file:/tmp/nds_q86_fail_test
java.lang.NullPointerException
at
org.apache.spark.util.JsonProtocol$JsonNodeImplicits.extractElements(JsonProtocol.scala:1589)
at
org.apache.spark.util.JsonProtocol$.stackTraceFromJson(JsonProtocol.scala:1558)
at
org.apache.spark.util.JsonProtocol$.exceptionFromJson(JsonProtocol.scala:1569)
at
org.apache.spark.util.JsonProtocol$.jobResultFromJson(JsonProtocol.scala:1423)
at
org.apache.spark.util.JsonProtocol$.jobEndFromJson(JsonProtocol.scala:967)
at
org.apache.spark.util.JsonProtocol$.sparkEventFromJson(JsonProtocol.scala:878)
at
org.apache.spark.util.JsonProtocol$.sparkEventFromJson(JsonProtocol.scala:865)
at
org.apache.spark.scheduler.ReplayListenerBus.replay(ReplayListenerBus.scala:88)
at
org.apache.spark.scheduler.ReplayListenerBus.replay(ReplayListenerBus.scala:59)
at
org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$3(FsHistoryProvider.scala:1140)
at
org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$3$adapted(FsHistoryProvider.scala:1138)
at org.apache.spark.util.Utils$.tryWithResource(Utils.scala:2786)
at
org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$1(FsHistoryProvider.scala:1138)
at
org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$1$adapted(FsHistoryProvider.scala:1136)
at scala.collection.immutable.List.foreach(List.scala:431)
at
org.apache.spark.deploy.history.FsHistoryProvider.parseAppEventLogs(FsHistoryProvider.scala:1136)
at
org.apache.spark.deploy.history.FsHistoryProvider.rebuildAppStore(FsHistoryProvider.scala:1117)
at
org.apache.spark.deploy.history.FsHistoryProvider.createInMemoryStore(FsHistoryProvider.scala:1355)
at
org.apache.spark.deploy.history.FsHistoryProvider.getAppUI(FsHistoryProvider.scala:345)
at
org.apache.spark.deploy.history.HistoryServer.getAppUI(HistoryServer.scala:199)
at
org.apache.spark.deploy.history.ApplicationCache.$anonfun$loadApplicationEntry$2(ApplicationCache.scala:163)
at
org.apache.spark.deploy.history.ApplicationCache.time(ApplicationCache.scala:134)
at
org.apache.spark.deploy.history.ApplicationCache.org$apache$spark$deploy$history$ApplicationCache$$loadApplicationEntry(ApplicationCache.scala:161)
at
org.apache.spark.deploy.history.ApplicationCache$$anon$1.load(ApplicationCache.scala:55)
at
org.apache.spark.deploy.history.ApplicationCache$$anon$1.load(ApplicationCache.scala:51)
at
org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
at
org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
at
org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
at org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257)
at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000)
at org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004)
at
org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
at
org.apache.spark.deploy.history.ApplicationCache.get(ApplicationCache.scala:88)
at
org.apache.spark.deploy.history.ApplicationCache.withSparkUI(ApplicationCache.scala:100)
at
org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$$loadAppUi(HistoryServer.scala:256)
at
org.apache.spark.deploy.history.HistoryServer$$anon$1.doGet(HistoryServer.scala:104)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:503)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:590)
at
org.sparkproject.jetty.servlet.ServletHolder.handle(ServletHolder.java:799)
at
org.sparkproject.jetty.servlet.ServletHandler$ChainEnd.doFilter(ServletHandler.java:1656)
at
org.apache.spark.ui.HttpSecurityFilter.doFilter(HttpSecurityFilter.scala:95)
at
org.sparkproject.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
at
org.sparkproject.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1626)
at
org.sparkproject.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:552)
at
org.sparkproject.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:233)
at
org.sparkproject.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1440)
at
org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:188)
at
org.sparkproject.jetty.servlet.ServletHandler.doScope(ServletHandler.java:505)
at
org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:186)
at
org.sparkproject.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1355)
at
org.sparkproject.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at
org.sparkproject.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:772)
at
org.sparkproject.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:234)
at
org.sparkproject.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:127)
at org.sparkproject.jetty.server.Server.handle(Server.java:516)
at
org.sparkproject.jetty.server.HttpChannel.lambda$handle$1(HttpChannel.java:487)
at org.sparkproject.jetty.server.HttpChannel.dispatch(HttpChannel.java:732)
at org.sparkproject.jetty.server.HttpChannel.handle(HttpChannel.java:479)
at
org.sparkproject.jetty.server.HttpConnection.onFillable(HttpConnection.java:277)
at
org.sparkproject.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:311)
at org.sparkproject.jetty.io.FillInterest.fillable(FillInterest.java:105)
at org.sparkproject.jetty.io.ChannelEndPoint$1.run(ChannelEndPoint.java:104)
at
org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:338)
at
org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:315)
at
org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:173)
at
org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:131)
at
org.sparkproject.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:409)
at
org.sparkproject.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:883)
at
org.sparkproject.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1034)
at java.lang.Thread.run(Thread.java:750)
23/05/01 21:57:17 ERROR ReplayListenerBus: Malformed line #24368:
\{"Event":"SparkListenerJobEnd","Job ID":31,"Completion
Time":1616171909785,"Job
Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted"}}}
23/05/01 21:57:17 INFO FsHistoryProvider: Finished parsing
file:/tmp/nds_q86_fail_test
```
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]