[jira] [Commented] (SPARK-19741) ClassCastException when using Dataset with type containing value types
[ https://issues.apache.org/jira/browse/SPARK-19741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15891548#comment-15891548 ] Kazuaki Ishizaki commented on SPARK-19741: -- I am afraid whether my sample program succeeded to reproduce the original issue. This is because the title of this PR says {{ClassCastException}} while I got another exception. > ClassCastException when using Dataset with type containing value types > -- > > Key: SPARK-19741 > URL: https://issues.apache.org/jira/browse/SPARK-19741 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL >Affects Versions: 2.1.0 > Environment: JDK 8 on Ubuntu > Scala 2.11.8 > Spark 2.1.0 >Reporter: Lior Regev > > The following code reproduces the error > {code} > final case class Foo(id: Int) extends AnyVal > final case class Bar(foo: Foo) > val foo = Foo(5) > val bar = Bar(foo) > import spark.implicits._ > spark.sparkContext.parallelize(0 to 10).toDS().map(_ => bar).collect() > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-19741) ClassCastException when using Dataset with type containing value types
[ https://issues.apache.org/jira/browse/SPARK-19741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15891463#comment-15891463 ] Hyukjin Kwon commented on SPARK-19741: -- I just tried the code above in the current master and it seems still not working whereas the reproducer in SPARK-17368 seems working fine. I tested both via IDE not Spark REPL. Apparently, it seems both JIRAs are similar. Maybe, we should make this as a duplicate or describe the differences between them in more details. > ClassCastException when using Dataset with type containing value types > -- > > Key: SPARK-19741 > URL: https://issues.apache.org/jira/browse/SPARK-19741 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL >Affects Versions: 2.1.0 > Environment: JDK 8 on Ubuntu > Scala 2.11.8 > Spark 2.1.0 >Reporter: Lior Regev > > The following code reproduces the error > {code} > final case class Foo(id: Int) extends AnyVal > final case class Bar(foo: Foo) > val foo = Foo(5) > val bar = Bar(foo) > import spark.implicits._ > spark.sparkContext.parallelize(0 to 10).toDS().map(_ => bar).collect() > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-19741) ClassCastException when using Dataset with type containing value types
[ https://issues.apache.org/jira/browse/SPARK-19741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15891348#comment-15891348 ] Sean Owen commented on SPARK-19741: --- Duplicate of SPARK-17368 ? > ClassCastException when using Dataset with type containing value types > -- > > Key: SPARK-19741 > URL: https://issues.apache.org/jira/browse/SPARK-19741 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL >Affects Versions: 2.1.0 > Environment: JDK 8 on Ubuntu > Scala 2.11.8 > Spark 2.1.0 >Reporter: Lior Regev > > The following code reproduces the error > {code} > final case class Foo(id: Int) extends AnyVal > final case class Bar(foo: Foo) > val foo = Foo(5) > val bar = Bar(foo) > import spark.implicits._ > spark.sparkContext.parallelize(0 to 10).toDS().map(_ => bar).collect() > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-19741) ClassCastException when using Dataset with type containing value types
[ https://issues.apache.org/jira/browse/SPARK-19741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15890712#comment-15890712 ] Kazuaki Ishizaki commented on SPARK-19741: -- The following program causes an exception regarding compilation error in Janino using the latest Spark master branch. {code:java} final case class Foo(id: Int) extends AnyVal final case class Bar(foo: Foo) object SPARK19741 { def main(args: Array[String]): Unit = { val foo = Foo(5) val bar = Bar(foo) val conf = new SparkConf().setAppName("test").setMaster("local") val spark = SparkSession.builder.config(conf)getOrCreate import spark.implicits._ spark.sparkContext.parallelize(0 to 10).toDS().map(_ => bar).collect() } } {code} Exception {code:java} 03:06:23.080 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 75, Column 29: Assignment conversion not possible from type "int" to type "org.apache.spark.sql.Foo" /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private int mapelements_argValue; /* 010 */ private UnsafeRow mapelements_result; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder; /* 012 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter; /* 013 */ private Object[] serializefromobject_values; /* 014 */ private UnsafeRow serializefromobject_result; /* 015 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 017 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter1; /* 018 */ /* 019 */ public GeneratedIterator(Object[] references) { /* 020 */ this.references = references; /* 021 */ } /* 022 */ /* 023 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 024 */ partitionIndex = index; /* 025 */ this.inputs = inputs; /* 026 */ inputadapter_input = inputs[0]; /* 027 */ /* 028 */ mapelements_result = new UnsafeRow(1); /* 029 */ this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32); /* 030 */ this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1); /* 031 */ this.serializefromobject_values = null; /* 032 */ serializefromobject_result = new UnsafeRow(1); /* 033 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32); /* 034 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 035 */ this.serializefromobject_rowWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 036 */ /* 037 */ } /* 038 */ /* 039 */ protected void processNext() throws java.io.IOException { /* 040 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 041 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 042 */ int inputadapter_value = inputadapter_row.getInt(0); /* 043 */ /* 044 */ boolean mapelements_isNull = true; /* 045 */ org.apache.spark.sql.Bar mapelements_value = null; /* 046 */ if (!false) { /* 047 */ mapelements_argValue = inputadapter_value; /* 048 */ /* 049 */ mapelements_isNull = false; /* 050 */ if (!mapelements_isNull) { /* 051 */ Object mapelements_funcResult = null; /* 052 */ mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue); /* 053 */ if (mapelements_funcResult == null) { /* 054 */ mapelements_isNull = true; /* 055 */ } else { /* 056 */ mapelements_value = (org.apache.spark.sql.Bar) mapelements_funcResult; /* 057 */ } /* 058 */ /* 059 */ } /* 060 */ mapelements_isNull = mapelements_value == null; /* 061 */ } /* 062 */ /* 063 */ if (mapelements_isNull) { /* 064 */ throw new RuntimeException(((java.lang.String) references[1])); /* 065 */ } /* 066 */ /* 067 */ if (false) { /* 068 */ throw new RuntimeException(((java.lang.String) references[2]));
[jira] [Commented] (SPARK-19741) ClassCastException when using Dataset with type containing value types
[ https://issues.apache.org/jira/browse/SPARK-19741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15885171#comment-15885171 ] Kazuaki Ishizaki commented on SPARK-19741: -- Would it be possible to attache a pair of the error message and stack trace? > ClassCastException when using Dataset with type containing value types > -- > > Key: SPARK-19741 > URL: https://issues.apache.org/jira/browse/SPARK-19741 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL >Affects Versions: 2.1.0 > Environment: JDK 8 on Ubuntu > Scala 2.11.8 > Spark 2.1.0 >Reporter: Lior Regev > > The following code reproduces the error > {code} > final case class Foo(id: Int) extends AnyVal > final case class Bar(foo: Foo) > val foo = Foo(5) > val bar = Bar(foo) > import spark.implicits._ > spark.sparkContext.parallelize(0 to 10).toDS().map(_ => bar).collect() > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org