[ https://issues.apache.org/jira/browse/SPARK-3630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14208908#comment-14208908 ]
Josh Rosen commented on SPARK-3630: ----------------------------------- Hi [~rdub], Thanks for posting those extra logs; I'm able to view them. I've been looking at the stacktraces from these logs and the logs that you sent earlier and it looks like you might actually be running an earlier version of Spark, not a recent version of 1.2. Many of your stacktraces refer to functionality that's been removed or renamed in recent-ish 1.2 builds. For example, in {{logs.2000.tt}}, the stracktrace {code} 14/11/12 00:32:34 WARN TaskSetManager: Lost task 654.0 in stage 2.0 (TID 5234, demeter-csmau08-11.demeter.hpc.mssm.edu): java.io.IOException: FAILED_TO_UNCOMPRESS(5) org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:84) org.xerial.snappy.SnappyNative.rawUncompress(Native Method) org.xerial.snappy.Snappy.rawUncompress(Snappy.java:444) org.xerial.snappy.Snappy.uncompress(Snappy.java:480) org.xerial.snappy.SnappyInputStream.readFully(SnappyInputStream.java:127) org.xerial.snappy.SnappyInputStream.readHeader(SnappyInputStream.java:88) org.xerial.snappy.SnappyInputStream.<init>(SnappyInputStream.java:58) org.apache.spark.io.SnappyCompressionCodec.compressedInputStream(CompressionCodec.scala:128) org.apache.spark.storage.BlockManager.wrapForCompression(BlockManager.scala:1090) org.apache.spark.storage.BlockManager.getLocalShuffleFromDisk(BlockManager.scala:350) org.apache.spark.storage.ShuffleBlockFetcherIterator$$anonfun$fetchLocalBlocks$1$$anonfun$apply$4.apply(ShuffleBlockFetcherIterator.scala:196) org.apache.spark.storage.ShuffleBlockFetcherIterator$$anonfun$fetchLocalBlocks$1$$anonfun$apply$4.apply(ShuffleBlockFetcherIterator.scala:196) org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:243) org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:52) scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:30) org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39) org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:89) org.apache.spark.shuffle.hash.HashShuffleReader.read(HashShuffleReader.scala:44) org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:92) org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) org.apache.spark.rdd.RDD.iterator(RDD.scala:229) org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31) org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) org.apache.spark.rdd.RDD.iterator(RDD.scala:229) org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) org.apache.spark.rdd.RDD.iterator(RDD.scala:229) org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61) org.apache.spark.scheduler.Task.run(Task.scala:56) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:181) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:744) {code} refers to {{org.apache.spark.storage.BlockManager.getLocalShuffleFromDisk(BlockManager.scala:350)}}, a function which isn't present in the version of Spark that you said you're using: https://github.com/apache/spark/blob/227488d8cd25d1ef0a53189adfd4797bf01fdf3c/core/src/main/scala/org/apache/spark/storage/BlockManager.scala Could you verify that your recent build of Spark has been properly deployed across your entire cluster and that both your Spark driver and workers are using the same Spark build? I'd be happy to hop on the phone / chat / video to help you debug this; you can get ahold of me at {{joshro...@databricks.com}}. > Identify cause of Kryo+Snappy PARSING_ERROR > ------------------------------------------- > > Key: SPARK-3630 > URL: https://issues.apache.org/jira/browse/SPARK-3630 > Project: Spark > Issue Type: Task > Components: Spark Core > Affects Versions: 1.1.0, 1.2.0 > Reporter: Andrew Ash > Assignee: Josh Rosen > > A recent GraphX commit caused non-deterministic exceptions in unit tests so > it was reverted (see SPARK-3400). > Separately, [~aash] observed the same exception stacktrace in an > application-specific Kryo registrator: > {noformat} > com.esotericsoftware.kryo.KryoException: java.io.IOException: failed to > uncompress the chunk: PARSING_ERROR(2) > com.esotericsoftware.kryo.io.Input.fill(Input.java:142) > com.esotericsoftware.kryo.io.Input.require(Input.java:169) > com.esotericsoftware.kryo.io.Input.readInt(Input.java:325) > com.esotericsoftware.kryo.io.Input.readFloat(Input.java:624) > com.esotericsoftware.kryo.serializers.DefaultSerializers$FloatSerializer.read(DefaultSerializers.java:127) > > com.esotericsoftware.kryo.serializers.DefaultSerializers$FloatSerializer.read(DefaultSerializers.java:117) > > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:732) > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:109) > > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:18) > > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:732) > ... > {noformat} > This ticket is to identify the cause of the exception in the GraphX commit so > the faulty commit can be fixed and merged back into master. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org