[ https://issues.apache.org/jira/browse/SPARK-23139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16356989#comment-16356989 ]
Imran Rashid commented on SPARK-23139: -------------------------------------- I think some confusion may come from the jira title & description -- its not mixed encoding within one event log file. Its that the directory of event logs have mixed encodings -- some with one encoding, some with another, based on whatever {{file.encoding}} was used with the spark app. I agree with Marcelo that we should just be enforcing UTF-8 regardless of the jvm's {{file.encoding}}. Looks like we're not when writing: https://github.com/apache/spark/blob/a75f927173632eee1316879447cb62c8cf30ae37/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala#L130 nor when reading https://github.com/apache/spark/blob/a75f927173632eee1316879447cb62c8cf30ae37/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala#L57 [~Deng FEI] do you want to update that as well? > Read eventLog file with mixed encodings > --------------------------------------- > > Key: SPARK-23139 > URL: https://issues.apache.org/jira/browse/SPARK-23139 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.2.0 > Reporter: DENG FEI > Priority: Major > > EventLog may contain mixed encodings such as custom exception message, but > caused to replay failure. > java.nio.charset.MalformedInputException: Input length = 1 > at java.nio.charset.CoderResult.throwException(CoderResult.java:281) > at sun.nio.cs.StreamDecoder.implRead(StreamDecoder.java:339) > at sun.nio.cs.StreamDecoder.read(StreamDecoder.java:178) > at java.io.InputStreamReader.read(InputStreamReader.java:184) > at java.io.BufferedReader.fill(BufferedReader.java:161) > at java.io.BufferedReader.readLine(BufferedReader.java:324) > at java.io.BufferedReader.readLine(BufferedReader.java:389) > at > scala.io.BufferedSource$BufferedLineIterator.hasNext(BufferedSource.scala:72) > at scala.collection.Iterator$$anon$21.hasNext(Iterator.scala:836) > at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:461) > at > org.apache.spark.scheduler.ReplayListenerBus.replay(ReplayListenerBus.scala:78) > at > org.apache.spark.scheduler.ReplayListenerBus.replay(ReplayListenerBus.scala:58) > at > org.apache.spark.deploy.history.FsHistoryProvider.org$apache$spark$deploy$history$FsHistoryProvider$$replay(FsHistoryProvider.scala:694) > at > org.apache.spark.deploy.history.FsHistoryProvider.org$apache$spark$deploy$history$FsHistoryProvider$$mergeApplicationListing(FsHistoryProvider.scala:507) > at > org.apache.spark.deploy.history.FsHistoryProvider$$anonfun$checkForLogs$4$$anon$4.run(FsHistoryProvider.scala:399) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org