You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Reynold Xin (JIRA)" <ji...@apache.org> on 2014/05/26 22:22:01 UTC

[jira] [Updated] (SPARK-1577) GraphX mapVertices with KryoSerialization

     [ https://issues.apache.org/jira/browse/SPARK-1577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Reynold Xin updated SPARK-1577:
-------------------------------

    Assignee: Ankur Dave

> GraphX mapVertices with KryoSerialization
> -----------------------------------------
>
>                 Key: SPARK-1577
>                 URL: https://issues.apache.org/jira/browse/SPARK-1577
>             Project: Spark
>          Issue Type: Bug
>          Components: GraphX
>            Reporter: Joseph E. Gonzalez
>            Assignee: Ankur Dave
>             Fix For: 1.0.0
>
>
> If Kryo is enabled by setting:
> {code}
> SPARK_JAVA_OPTS+="-Dspark.serializer=org.apache.spark.serializer.KryoSerializer "
> SPARK_JAVA_OPTS+="-Dspark.kryo.registrator=org.apache.spark.graphx.GraphKryoRegistrator  "
> {code}
> in conf/spark_env.conf and running the following block of code in the shell:
> {code}
> import org.apache.spark.graphx._
> import org.apache.spark.graphx.lib._
> import org.apache.spark.rdd.RDD
> val vertexArray = Array(
>   (1L, ("Alice", 28)),
>   (2L, ("Bob", 27)),
>   (3L, ("Charlie", 65)),
>   (4L, ("David", 42)),
>   (5L, ("Ed", 55)),
>   (6L, ("Fran", 50))
>   )
> val edgeArray = Array(
>   Edge(2L, 1L, 7),
>   Edge(2L, 4L, 2),
>   Edge(3L, 2L, 4),
>   Edge(3L, 6L, 3),
>   Edge(4L, 1L, 1),
>   Edge(5L, 2L, 2),
>   Edge(5L, 3L, 8),
>   Edge(5L, 6L, 3)
>   )
> val vertexRDD: RDD[(Long, (String, Int))] = sc.parallelize(vertexArray)
> val edgeRDD: RDD[Edge[Int]] = sc.parallelize(edgeArray)
> val graph: Graph[(String, Int), Int] = Graph(vertexRDD, edgeRDD)
> // Define a class to more clearly model the user property
> case class User(name: String, age: Int, inDeg: Int, outDeg: Int)
> // Transform the graph
> val userGraph = graph.mapVertices{ case (id, (name, age)) => User(name, age, 0, 0) }
> {code}
> The following block of code works:
> {code}
> userGraph.vertices.count
> {code}
> and the following block of code generates a Kryo error:
> {code}
> userGraph.vertices.collect
> {code}
> There error:
> {code}
> java.lang.StackOverflowError
> 	at sun.reflect.UnsafeFieldAccessorImpl.ensureObj(UnsafeFieldAccessorImpl.java:54)
> 	at sun.reflect.UnsafeQualifiedObjectFieldAccessorImpl.get(UnsafeQualifiedObjectFieldAccessorImpl.java:38)
> 	at java.lang.reflect.Field.get(Field.java:379)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.write(FieldSerializer.java:552)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:213)
> 	at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:501)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.write(FieldSerializer.java:564)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:213)
> 	at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:501)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.write(FieldSerializer.java:564)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:213)
> 	at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:501)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer$ObjectField.write(FieldSerializer.java:564)
> 	at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:213)
> 	at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:501)
> {code}
>  



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