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

Thomas Graves commented on SPARK-10735:
---------------------------------------

They are using SQLContext.createDataFrame(RDD<T>, Class<T>)  with the bean 
classes.   Beacon is a user defined type but I don't see it being registered 
with UserDefinedType.  Beacon is a simple class with an int and a long as 
members.  

If its registered there would you expect it to work?

I don't see any docs on UserDefinedType usage (other then scala docs), pointers 
on simple usage?  I assume its just annotate with the SQLUserDefinedType and 
then write the class to do serialize, deserialize ,etc... If this is the case 
perhaps we need to modify the section about 
http://spark.apache.org/docs/latest/sql-programming-guide.html#interoperating-with-rdds

> CatalystTypeConverters MatchError converting RDD with custom object to 
> dataframe
> --------------------------------------------------------------------------------
>
>                 Key: SPARK-10735
>                 URL: https://issues.apache.org/jira/browse/SPARK-10735
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.5.0
>            Reporter: Thomas Graves
>            Priority: Critical
>
> In spark 1.5.0 we are now seeing an exception when converting an RDD with 
> custom object to a dataframe.  Note this works with Spark 1.4.1.
> RDD<BasicData>
> where BasicData class has a field ArrayList<Beacon> where Beacon is a user 
> defined class now converting RDD<BasicData> to DataFrame is causing the issue:
> {code}
> 15/09/21 18:53:16 ERROR executor.Executor: Managed memory leak detected; size 
> = 2097152 bytes, TID = 408
> 15/09/21 18:53:16 ERROR executor.Executor: Exception in task 0.0 in stage 4.0 
> (TID 408)
> scala.MatchError: foo.Beacon@5c289b39 (of class foo.Beacon)
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:250)
>   
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:245)
>   
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$ArrayConverter.toCatalystImpl(CatalystTypeConverters.scala:164)
>    
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$ArrayConverter.toCatalystImpl(CatalystTypeConverters.scala:148)
>    
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>         at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:396)
>         at 
> org.apache.spark.sql.SQLContext$$anonfun$9$$anonfun$apply$1$$anonfun$apply$2.apply(SQLContext.scala:494)
>         at 
> org.apache.spark.sql.SQLContext$$anonfun$9$$anonfun$apply$1$$anonfun$apply$2.apply(SQLContext.scala:494)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>         at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>         at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108)
>         at 
> scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
>         at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:108)
>         at 
> org.apache.spark.sql.SQLContext$$anonfun$9$$anonfun$apply$1.apply(SQLContext.scala:494)
>         at 
> org.apache.spark.sql.SQLContext$$anonfun$9$$anonfun$apply$1.apply(SQLContext.scala:492)
>         at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>         at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.processInputs(TungstenAggregationIterator.scala:372)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.start(TungstenAggregationIterator.scala:622)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.org$apache$spark$sql$execution$aggregate$TungstenAggregate$$anonfun$$executePartition$1(TungstenAggregate.scala:110)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
>         at 
> org.apache.spark.rdd.MapPartitionsWithPreparationRDD.compute(MapPartitionsWithPreparationRDD.scala:64)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>         at org.apache.spark.scheduler.Task.run(Task.scala:88)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>         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)
> {code}



--
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

Reply via email to