Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
3.5.3
-
None
Description
There is an issue with the Spark History UI with large amounts of event logs.
The root of this problem is the breaking change in jackson that (in the name of "safety") introduced some JSON size limits, see: https://github.com/FasterXML/jackson-core/issues/1014
Looks like JSONOptions in Spark already support configuring this limit, but there seems to be no way to set it globally.
Spark should be able to handle strings of arbitrary length. I have tried configuring rolling event logs, pruning event logs, etc. but this issue is not fixed or causes so much data loss that the spark history ui is completely useless.
Perhaps a solution could be to add a config like:
spark.history.server.jsonStreamReadConstraints.maxStringLength=<new_value>
This has a workaround for reading JSON during your application:
spark.read.option("maxStringLen", 100000000).json(path)
But this is not an option for accessing the Spark History UI. Here is the full stack trace
HTTP ERROR 500 com.fasterxml.jackson.core.exc.StreamConstraintsException: String length (20054016) exceeds the maximum length (20000000) URI:/history/application_1728009195451_0002/1/jobs/ STATUS:500 MESSAGE:com.fasterxml.jackson.core.exc.StreamConstraintsException: String length (20054016) exceeds the maximum length (20000000) SERVLET:org.apache.spark.deploy.history.HistoryServer$$anon$1-582a764a CAUSED BY:com.fasterxml.jackson.core.exc.StreamConstraintsException: String length (20054016) exceeds the maximum length (20000000) com.fasterxml.jackson.core.exc.StreamConstraintsException: String length (20054016) exceeds the maximum length (20000000) at com.fasterxml.jackson.core.StreamReadConstraints.validateStringLength(StreamReadConstraints.java:324) at com.fasterxml.jackson.core.util.ReadConstrainedTextBuffer.validateStringLength(ReadConstrainedTextBuffer.java:27) at com.fasterxml.jackson.core.util.TextBuffer.finishCurrentSegment(TextBuffer.java:939) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser._finishString2(ReaderBasedJsonParser.java:2240) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser._finishString(ReaderBasedJsonParser.java:2206) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser.getText(ReaderBasedJsonParser.java:323) at com.fasterxml.jackson.databind.deser.std.BaseNodeDeserializer._deserializeContainerNoRecursion(JsonNodeDeserializer.java:572) at com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer.deserialize(JsonNodeDeserializer.java:100) at com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer.deserialize(JsonNodeDeserializer.java:25) at com.fasterxml.jackson.databind.deser.DefaultDeserializationContext.readRootValue(DefaultDeserializationContext.java:323) at com.fasterxml.jackson.databind.ObjectMapper._readTreeAndClose(ObjectMapper.java:4867) at com.fasterxml.jackson.databind.ObjectMapper.readTree(ObjectMapper.java:3219) at org.apache.spark.util.JsonProtocol$.sparkEventFromJson(JsonProtocol.scala:927) 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:1143) at org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$3$adapted(FsHistoryProvider.scala:1141) at org.apache.spark.util.SparkErrorUtils.tryWithResource(SparkErrorUtils.scala:48) at org.apache.spark.util.SparkErrorUtils.tryWithResource$(SparkErrorUtils.scala:46) at org.apache.spark.util.Utils$.tryWithResource(Utils.scala:95) at org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$1(FsHistoryProvider.scala:1141) at org.apache.spark.deploy.history.FsHistoryProvider.$anonfun$parseAppEventLogs$1$adapted(FsHistoryProvider.scala:1139) at scala.collection.immutable.List.foreach(List.scala:431) at org.apache.spark.deploy.history.FsHistoryProvider.parseAppEventLogs(FsHistoryProvider.scala:1139) at org.apache.spark.deploy.history.FsHistoryProvider.rebuildAppStore(FsHistoryProvider.scala:1120) at org.apache.spark.deploy.history.FsHistoryProvider.createInMemoryStore(FsHistoryProvider.scala:1358) at org.apache.spark.deploy.history.FsHistoryProvider.getAppUI(FsHistoryProvider.scala:347) 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.apache.spark.ui.ProxyRedirectHandler.handle(JettyUtils.scala:582) 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)
Attachments
Issue Links
- links to