You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Gengliang Wang (Jira)" <ji...@apache.org> on 2022/05/10 15:11:00 UTC

[jira] [Created] (SPARK-39140) JavaSerializer doesn't serialize the fields of superclass

Gengliang Wang created SPARK-39140:
--------------------------------------

             Summary: JavaSerializer doesn't serialize the fields of superclass
                 Key: SPARK-39140
                 URL: https://issues.apache.org/jira/browse/SPARK-39140
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 3.3.0
            Reporter: Gengliang Wang


To reproduce:

 
{code:java}
abstract class AA {
  val ts = System.nanoTime()
}
case class BB(x: Int) extends AA {
}

val input = BB(1)
println("original ts: " + input.ts)

val javaSerializer = new JavaSerializer(new SparkConf())
val javaInstance = javaSerializer.newInstance()
val bytes1 = javaInstance.serialize[BB](input)
val obj1 = javaInstance.deserialize[BB](bytes1)
println("deserialization result from java: " + obj1.ts)

val kryoSerializer = new KryoSerializer(new SparkConf())
val kryoInstance = kryoSerializer.newInstance()
val bytes2 = kryoInstance.serialize[BB](input)
val obj2 = kryoInstance.deserialize[BB](bytes2)
println("deserialization result from kryo: " + obj2.ts) {code}
 

 

The output is

 
{code:java}
original ts: 115014173658666
deserialization result from java: 115014306794333
deserialization result from kryo: 115014173658666{code}
 

We can see that the fields from the superclass AA are not serialized with JavaSerializer. When switching to KryoSerializer, it works.

This caused bugs in the project SPARK-38615: TreeNode.origin is not serialized to executors when a plan can't be executed with whole-staged-codegen.

It could also lead to bugs in serializing the lambda function within RDD API like 

mapPartitions/mapPartitionsWithIndex/etc.

 



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org