[ 
https://issues.apache.org/jira/browse/SPARK-2610?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14069513#comment-14069513
 ] 

Yin Huai commented on SPARK-2610:
---------------------------------

For JavaSerializer...
{code}
class X() { println("What!"); def y = 3 }
val x = new X
import x.y

case class Person(name: String, age: Int)

val serializer = org.apache.spark.serializer.Serializer.getSerializer(null)
val javaSerializer = serializer.newInstance

val value = javaSerializer.serialize(Person("abc", 1))
javaSerializer.deserialize(value): Person
{code}

I got a NotSerializableException exception after I call serialize.
{code}
java.io.NotSerializableException: $iwC$$iwC$$iwC$$iwC$X
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at 
java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
        at 
java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
        at 
java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
        at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
        at 
org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:42)
        at 
org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:71)
        at $iwC$$iwC$$iwC$$iwC.<init>(<console>:20)
        at $iwC$$iwC$$iwC.<init>(<console>:25)
        at $iwC$$iwC.<init>(<console>:27)
        at $iwC.<init>(<console>:29)
        at <init>(<console>:31)
        at .<init>(<console>:35)
        at .<clinit>(<console>)
        at .<init>(<console>:7)
        at .<clinit>(<console>)
        at $print(<console>)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:788)
        at 
org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1056)
        at 
org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:614)
        at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:645)
        at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:609)
        at 
org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:796)
        at 
org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:841)
        at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:753)
        at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:601)
        at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:608)
        at org.apache.spark.repl.SparkILoop.loop(SparkILoop.scala:611)
        at 
org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:936)
        at 
org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884)
        at 
org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884)
        at 
scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
        at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:884)
        at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:982)
        at org.apache.spark.repl.Main$.main(Main.scala:31)
        at org.apache.spark.repl.Main.main(Main.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:303)
        at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:55)
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
{code}

Seems REPL pulls in some unnecessary things and Kryo can serialize more things 
than Java serializer.

> When spark.serializer is set as org.apache.spark.serializer.KryoSerializer, 
> importing a method causes multiple spark applications creations  
> ---------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-2610
>                 URL: https://issues.apache.org/jira/browse/SPARK-2610
>             Project: Spark
>          Issue Type: Bug
>    Affects Versions: 1.0.1
>            Reporter: Yin Huai
>
> To reproduce, set
> {code}
> spark.serializer        org.apache.spark.serializer.KryoSerializer
> {code}
> in conf/spark-defaults.conf and launch a spark shell.
> Then, execute
> {code}
> class X() { println("What!"); def y = 3 }
> val x = new X
> import x.y
> case class Person(name: String, age: Int)
> val serializer = org.apache.spark.serializer.Serializer.getSerializer(null)
> val kryoSerializer = serializer.newInstance
> val value = kryoSerializer.serialize(Person("abc", 1))
> kryoSerializer.deserialize(value): Person
> // Once you execute this line, you will see ...
> // What!
> // What!
> // res1: Person = Person(abc,1)
> {code}
> Basically, importing a method of a class causes the constructor of that class 
> been called twice.
> It affects our branch 1.0 and master.
> For the master, you can use 
> {code}
> val serializer = org.apache.spark.serializer.Serializer.getSerializer(None)
> {code}
> to get the serializer.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to