[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2018-08-12 Thread Minh Thai (JIRA)


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

Minh Thai commented on SPARK-17368:
---

[~jodersky] I know that this is an old ticket but I still want to give some 
comments on making encoder for value classes. Even until today, there is no way 
to have a type constraint that targets value classes. However, I think we can 
make a [universal 
trait|https://docs.scala-lang.org/overviews/core/value-classes.html] called 
{{OpaqueValue}}^1^ to be used as an upper type bound in encoder. This means:
 - Any user-defined value class has to mixin {{OpaqueValue}}
 - An encoder can be created to target those value classes.

{code}
trait OpaqueValue extends Any
implicit def newValueClassEncoder[T <: Product with OpaqueValue : TypeTag] = ???

case class Id(value: Int) extends AnyVal with OpaqueValue
{code}
tested on my machine using Spark 2.1.0 and Scala 2.11.12, this doesn't clash 
with the existing encoder for case class
{code}
implicit def newProductEncoder[T <: Product : TypeTag]: Encoder[T] = 
Encoders.product[T]
{code}
_(1) the name is inspired from [Opaque 
Type|https://docs.scala-lang.org/sips/opaque-types.html] feature of Scala 3_

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>Assignee: Jakob Odersky
>Priority: Major
> Fix For: 2.1.0
>
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-10-17 Thread Aris Vlasakakis (JIRA)

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

Aris Vlasakakis commented on SPARK-17368:
-

That is great, thank you for the help with this.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>Assignee: Jakob Odersky
> Fix For: 2.1.0
>
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-10-17 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

[~arisofala...@gmail.com] Let me explain the fix to what I thought was 
initially impossible.
Value classes do have a class-representation for compatibility with Java, and 
although this will have a slight overhead compared to the primitive 
counterpart, catalyst will mostly negate that overhead by proving its own 
encoders and operators on serialized objects. This means that any operations on 
datasets that allow user defined functions (e.g. `map`, `filter` etc) will work 
with the class representation instead of the wrapped value.
Regarding the availability of encoders: while we cannot create type-classes 
that apply only to value classes (an implicit for `AnyVal` will also be applied 
to primitive types), without resorting to macros, this fix adds value class 
support to existing encoders. E.g. you can define your value class as a case 
class and have a working encoder out-of-the-box.
Unfortunately there is no way to statically verify that the wrapped value is 
also encodable, but encoders in general will perform "deep inspection" during 
runtime.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>Assignee: Jakob Odersky
> Fix For: 2.1.0
>
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-28 Thread Apache Spark (JIRA)

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

Apache Spark commented on SPARK-17368:
--

User 'jodersky' has created a pull request for this issue:
https://github.com/apache/spark/pull/15284

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-07 Thread Sean Owen (JIRA)

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

Sean Owen commented on SPARK-17368:
---

This is beyond my knowledge I'm afraid. I'd help take a look if I can but not 
sure I'd know where to start on it myself!

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-07 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

Hmm, you're right my assumption was of using only value classes in the 
beginning and at the end was too naive.

[~srowen], how likely do you think it is that we can include a meta-encoder in 
Spark? It could be included in the form of an optional import. Since the 
existing encoders/ScalaReflection framework already use runtime-reflection, my 
guess is that adding compile-time reflection will not be too difficult.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-07 Thread Sean Owen (JIRA)

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

Sean Owen commented on SPARK-17368:
---

I get the problem, but is there actually any solution? this is a compile-time 
construct.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-06 Thread Aris Vlasakakis (JIRA)

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

Aris Vlasakakis commented on SPARK-17368:
-

It goes from inconvenient to actually prohibitive in a practical sense. I have 
a Dataset[Something], and inside case class Something I have various other case 
classes, and somewhere inside there there is a particular value class. It is so 
crazy to do manual unwrapping and rewrapping that at this point I just decided 
to eat the performance cost and use a regular class, not value class (I removed 
the 'extends AnyVal').

More generally, specially accommodating for value classes is *really hard* in a 
practical setting because if I have a whole bunch of ADTs and other case 
classes I'm working with, how do I know if anywhere in my domain I used a 
*value class* and I suddenly have to jump through a bunch of hoops just so 
Spark doesn't blow up? If I just had a Dataset[ThisIsAValueClass] with the 
top-level class being a value class, what you're saying is easy, but in 
practice the value class is one of many things somewhere deeper.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-06 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

So I thought about this a bit more and although it is possible to support value 
classes, I currently see two main issues that make it cumbersome:

1. Catalyst (the engine behind Datasets) generates and compiles code during 
runtime, that will represent the actual computation. This code being Java, 
together with the fact that value classes don't have runtime representations, 
will require changes in the implementation of Encoders (see my experimental 
branch here).

2. The largest problem of both is how will encoders for value classes be 
accessible? Currently, encoders are exposed as type classes and there is 
unfortunately no way to create type classes for classes extending AnyVal (you 
could create an encoder for AnyVals, however that would also apply to any 
primitive type and you would get implicit resolution conflicts). Requiring 
explicit encoders for value classes may work, however you would still have no 
compile-time safety, as accessing of a value class' inner val will occur during 
runtime and may hence fail if it is not encodable.

The cleanest solution would be to use meta programming: it would guarantee 
"encodability" during compile-time and could easily complement the current API. 
Unfortunately however, I don't think it could be included in Spark in the near 
future as the current meta programming solutions in Scala are either too new 
(scala.meta) or on their way to being deprecated (the current experimental 
scala macros). (I have been wanting to experiment with meta encoders for a 
while though, so maybe I'll try putting together an external library for that)

How inconvenient is it to extract the wrapped value before creating a dataset 
and re-wrapping your final results?

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-02 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

Yeah macros would be awesome, something with Scala.meta would be neat :) In the 
mean time it occurred to me that Catalyst uses ClassTags to do reflection in 
lots of places. These are generated during compile-time, so it might just yet 
be possible to support value classes.
A quick test showed me that value classes can be detected and their parameters 
accessed. Getting a Schema for such a case is trivial, I'll see about adding 
encoders next!

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-02 Thread Aris Vlasakakis (JIRA)

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

Aris Vlasakakis commented on SPARK-17368:
-

I actually had an identical first thought from my experience of value classes 
and how they disappear in the JVM byte code. It would be very helpful if in the 
documentation somewhere that it was said that Scala value classes were 
explicitly not supported by spark datasets. The error messages are extremely 
cryptic and very confusing. Even better would be some kind of macro support or 
whatever else by Spark that would find and call it out of your code, but that's 
wishful thinking.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-02 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

I'm currently taking a look at this but my first analysis is not very positive: 
considering that value classes are pure compile-time constructs I think it 
isn't possible to do anything with them through reflection, which Catalyst 
assumes.
Here's a relevant blog post 
http://tech.kinja.com/scala-value-classes-and-reflection-here-be-dragons-1527846740
I'll check it out in a bit more detail but I fear that we'll have to resolve 
this as a won't fix and not support value classes in Datasets :(

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-01 Thread Aris Vlasakakis (JIRA)

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

Aris Vlasakakis commented on SPARK-17368:
-

Yes, agreed that this code cannot be pasted into {{spark-shell}}. My assumption 
was that this would be a compiled JAR and then passed into {{spark-submit}} -- 
a normal Spark application. When you do so, the code compiles but Spark breaks 
at runtime.

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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



[jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime

2016-09-01 Thread Jakob Odersky (JIRA)

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

Jakob Odersky commented on SPARK-17368:
---

FYI the issue also occurs for top-level value classes (i.e. {{FeatureId}} 
defined outside of {{object BreakSpark}})

> Scala value classes create encoder problems and break at runtime
> 
>
> Key: SPARK-17368
> URL: https://issues.apache.org/jira/browse/SPARK-17368
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 1.6.2, 2.0.0
> Environment: JDK 8 on MacOS
> Scala 2.11.8
> Spark 2.0.0
>Reporter: Aris Vlasakakis
>
> Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 
> and 1.6.X.
> This simple Spark 2 application demonstrates that the code will compile, but 
> will break at runtime with the error. The value class is of course 
> *FeatureId*, as it extends AnyVal.
> {noformat}
> Exception in thread "main" java.lang.RuntimeException: Error while encoding: 
> java.lang.RuntimeException: Couldn't find v on int
> assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0
> +- assertnotnull(input[0, int, true], top level non-flat input object).v
>+- assertnotnull(input[0, int, true], top level non-flat input object)
>   +- input[0, int, true]".
> at 
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> at 
> org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421)
> {noformat}
> Test code for Spark 2.0.0:
> {noformat}
> import org.apache.spark.sql.{Dataset, SparkSession}
> object BreakSpark {
>   case class FeatureId(v: Int) extends AnyVal
>   def main(args: Array[String]): Unit = {
> val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3))
> val spark = SparkSession.builder.getOrCreate()
> import spark.implicits._
> spark.sparkContext.setLogLevel("warn")
> val ds: Dataset[FeatureId] = spark.createDataset(seq)
> println(s"BREAK HERE: ${ds.count}")
>   }
> }
> {noformat}



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