You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Alexander Lopatin (JIRA)" <ji...@apache.org> on 2017/07/17 10:41:01 UTC

[jira] [Commented] (SPARK-12777) Dataset fields can't be Scala tuples

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

Alexander Lopatin commented on SPARK-12777:
-------------------------------------------

The issue reproduces for me in standalone application with Spark 2.2.0 when the type is explicitly specified.

{code:scala}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.{SparkConf, SparkContext}

object Main extends App {

  val sc = new SparkContext("local", "hello", new SparkConf())
  val ss = SparkSession.builder().getOrCreate()

  import ss.implicits._

  val seq = Seq((1, (2, 3)))

  type A = (Int, Int)

  val rdd: RDD[(Int, A)] = sc.parallelize(seq)
  //val rdd: RDD[(Int, (Int, Int))] = sc.parallelize(seq) // works fine
  //val rdd = sc.parallelize(seq) // works fine

  ss.createDataset(rdd).show()

  sc.stop()
}
{code}

Stacktrace (Spark 2.2.0):
{code:java}
scala.ScalaReflectionException: type T1 is not a class
        at scala.reflect.api.Symbols$SymbolApi$class.asClass(Symbols.scala:275)
        at scala.reflect.internal.Symbols$SymbolContextApiImpl.asClass(Symbols.scala:84)
        at org.apache.spark.sql.catalyst.ScalaReflection$.getClassFromType(ScalaReflection.scala:682)
        at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor(ScalaReflection.scala:84)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$10.apply(ScalaReflection.scala:614)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$10.apply(ScalaReflection.scala:607)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.immutable.List.flatMap(List.scala:344)
        at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:607)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$10.apply(ScalaReflection.scala:619)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$10.apply(ScalaReflection.scala:607)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.immutable.List.flatMap(List.scala:344)
        at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:607)
        at org.apache.spark.sql.catalyst.ScalaReflection$.serializerFor(ScalaReflection.scala:438)
        at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:71)
        at org.apache.spark.sql.Encoders$.product(Encoders.scala:275)
        at org.apache.spark.sql.LowPrioritySQLImplicits$class.newProductEncoder(SQLImplicits.scala:233)
        at org.apache.spark.sql.SQLImplicits.newProductEncoder(SQLImplicits.scala:33)
        at Main$.delayedEndpoint$Main$1(Main.scala:20)
        at Main$delayedInit$body.apply(Main.scala:5)
        at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
        at scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
        at scala.App$$anonfun$main$1.apply(App.scala:76)
        at scala.App$$anonfun$main$1.apply(App.scala:76)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
        at scala.App$class.main(App.scala:76)
        at Main$.main(Main.scala:5)
        at Main.main(Main.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
{code}

Stacktrace (Spark 2.1.0):
{code:java}
java.util.NoSuchElementException: head of empty list
        at scala.collection.immutable.Nil$.head(List.scala:420)
        at scala.collection.immutable.Nil$.head(List.scala:417)
        at scala.reflect.internal.tpe.TypeMaps$SubstMap.subst(TypeMaps.scala:709)
        at scala.reflect.internal.tpe.TypeMaps$SubstMap.substFor$1(TypeMaps.scala:717)
        at scala.reflect.internal.tpe.TypeMaps$SubstMap.apply(TypeMaps.scala:732)
        at scala.reflect.internal.Types$Type.subst(Types.scala:705)
        at scala.reflect.internal.Types$TypeApiImpl.substituteTypes(Types.scala:240)
        at scala.reflect.internal.Types$TypeApiImpl.substituteTypes(Types.scala:218)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$getConstructorParameters$1.apply(ScalaReflection.scala:818)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$getConstructorParameters$1.apply(ScalaReflection.scala:817)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.immutable.List.map(List.scala:285)
        at org.apache.spark.sql.catalyst.ScalaReflection$class.getConstructorParameters(ScalaReflection.scala:817)
        at org.apache.spark.sql.catalyst.ScalaReflection$.getConstructorParameters(ScalaReflection.scala:39)
        at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:586)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:596)
        at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:587)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.immutable.List.flatMap(List.scala:344)
        at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:587)
        at org.apache.spark.sql.catalyst.ScalaReflection$.serializerFor(ScalaReflection.scala:425)
        at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:71)
        at org.apache.spark.sql.Encoders$.product(Encoders.scala:275)
        at org.apache.spark.sql.SQLImplicits.newProductEncoder(SQLImplicits.scala:49)
        at Main$.delayedEndpoint$Main$1(Main.scala:20)
        at Main$delayedInit$body.apply(Main.scala:5)
        at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
        at scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
        at scala.App$$anonfun$main$1.apply(App.scala:76)
        at scala.App$$anonfun$main$1.apply(App.scala:76)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
        at scala.App$class.main(App.scala:76)
        at Main$.main(Main.scala:5)
        at Main.main(Main.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
{code}

> Dataset fields can't be Scala tuples
> ------------------------------------
>
>                 Key: SPARK-12777
>                 URL: https://issues.apache.org/jira/browse/SPARK-12777
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.0, 1.6.1, 2.0.0
>            Reporter: Chris Jansen
>            Assignee: Jakob Odersky
>             Fix For: 2.1.0
>
>
> Datasets can't seem to handle scala tuples as fields of case classes in datasets.
> {code}
> Seq((1,2), (3,4)).toDS().show() //works
> {code}
> When including a tuple as a field, the code fails:
> {code}
> case class Test(v: (Int, Int))
> Seq(Test((1,2)), Test((3,4)).toDS().show //fails
> {code}
> {code}
>   UnresolvedException: : Invalid call to dataType on unresolved object, tree: 'name  (unresolved.scala:59)
>  org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:59)
>  org.apache.spark.sql.catalyst.expressions.GetStructField.org$apache$spark$sql$catalyst$expressions$GetStructField$$field$lzycompute(complexTypeExtractors.scala:107)
>  org.apache.spark.sql.catalyst.expressions.GetStructField.org$apache$spark$sql$catalyst$expressions$GetStructField$$field(complexTypeExtractors.scala:107)
>  org.apache.spark.sql.catalyst.expressions.GetStructField$$anonfun$toString$1.apply(complexTypeExtractors.scala:111)
>  org.apache.spark.sql.catalyst.expressions.GetStructField$$anonfun$toString$1.apply(complexTypeExtractors.scala:111)
>  org.apache.spark.sql.catalyst.expressions.GetStructField.toString(complexTypeExtractors.scala:111)
>  org.apache.spark.sql.catalyst.expressions.Expression.toString(Expression.scala:217)
>  org.apache.spark.sql.catalyst.expressions.Expression.toString(Expression.scala:217)
>  org.apache.spark.sql.catalyst.expressions.If.toString(conditionalExpressions.scala:76)
>  org.apache.spark.sql.catalyst.expressions.Expression.toString(Expression.scala:217)
>  org.apache.spark.sql.catalyst.expressions.Alias.toString(namedExpressions.scala:155)
>  org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$argString$1.apply(TreeNode.scala:385)
>  org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$argString$1.apply(TreeNode.scala:381)
>  org.apache.spark.sql.catalyst.trees.TreeNode.argString(TreeNode.scala:388)
>  org.apache.spark.sql.catalyst.trees.TreeNode.simpleString(TreeNode.scala:391)
>  org.apache.spark.sql.catalyst.plans.QueryPlan.simpleString(QueryPlan.scala:172)
>  org.apache.spark.sql.catalyst.trees.TreeNode.generateTreeString(TreeNode.scala:441)
>  org.apache.spark.sql.catalyst.trees.TreeNode.treeString(TreeNode.scala:396)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$5.apply(RuleExecutor.scala:118)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$5.apply(RuleExecutor.scala:119)
>  org.apache.spark.Logging$class.logDebug(Logging.scala:62)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor.logDebug(RuleExecutor.scala:44)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:115)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:72)
>  org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:72)
>  org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.resolve(ExpressionEncoder.scala:253)
>  org.apache.spark.sql.Dataset.<init>(Dataset.scala:78)
>  org.apache.spark.sql.Dataset.<init>(Dataset.scala:89)
>  org.apache.spark.sql.SQLContext.createDataset(SQLContext.scala:507)
>  org.apache.spark.sql.SQLImplicits.localSeqToDatasetHolder(SQLImplicits.scala:80)
> {code}
> When providing a type alias, the code fails in a different way:
> {code}
> type TwoInt = (Int, Int)
> case class Test(v: TwoInt)
> Seq(Test((1,2)), Test((3,4)).toDS().show //fails
> {code}
> {code}
>   NoSuchElementException: : head of empty list  (ScalaReflection.scala:504)
>  org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor$1.apply(ScalaReflection.scala:504)
>  org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor$1.apply(ScalaReflection.scala:502)
>  org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor(ScalaReflection.scala:502)
>  org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor$1.apply(ScalaReflection.scala:509)
>  org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor$1.apply(ScalaReflection.scala:502)
>  org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$extractorFor(ScalaReflection.scala:502)
>  org.apache.spark.sql.catalyst.ScalaReflection$.extractorsFor(ScalaReflection.scala:394)
>  org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:54)
>  org.apache.spark.sql.SQLImplicits.newProductEncoder(SQLImplicits.scala:41)
>  com.intenthq.pipeline.actions.ActionsJobIntegrationSpec.enrich(ActionsJobIntegrationSpec.scala:63)
>  com.intenthq.pipeline.actions.ActionsJobIntegrationSpec$$anonfun$is$2.apply(ActionsJobIntegrationSpec.scala:45)
>  com.intenthq.pipeline.actions.ActionsJobIntegrationSpec$$anonfun$is$2.apply(ActionsJobIntegrationSpec.scala:45)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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