You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Kazuaki Ishizaki (JIRA)" <ji...@apache.org> on 2017/03/01 18:12:45 UTC
[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&focusedCommentId=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]));
/* 069 */ }
/* 070 */ boolean serializefromobject_isNull2 = true;
/* 071 */ org.apache.spark.sql.Foo serializefromobject_value2 = null;
/* 072 */ if (!false) {
/* 073 */ serializefromobject_isNull2 = false;
/* 074 */ if (!serializefromobject_isNull2) {
/* 075 */ serializefromobject_value2 = mapelements_value.foo();
/* 076 */ }
/* 077 */ serializefromobject_isNull2 = serializefromobject_value2 == null;
/* 078 */ }
/* 079 */ boolean serializefromobject_isNull = false;
/* 080 */ InternalRow serializefromobject_value = null;
/* 081 */ if (!false && serializefromobject_isNull2) {
/* 082 */ final InternalRow serializefromobject_value6 = null;
/* 083 */ serializefromobject_isNull = true;
/* 084 */ serializefromobject_value = serializefromobject_value6;
/* 085 */ } else {
/* 086 */ serializefromobject_values = new Object[1];
/* 087 */ if (mapelements_isNull) {
/* 088 */ throw new RuntimeException(((java.lang.String) references[3]));
/* 089 */ }
/* 090 */
/* 091 */ if (false) {
/* 092 */ throw new RuntimeException(((java.lang.String) references[4]));
/* 093 */ }
/* 094 */ boolean serializefromobject_isNull10 = true;
/* 095 */ org.apache.spark.sql.Foo serializefromobject_value10 = null;
/* 096 */ if (!false) {
/* 097 */ serializefromobject_isNull10 = false;
/* 098 */ if (!serializefromobject_isNull10) {
/* 099 */ serializefromobject_value10 = mapelements_value.foo();
/* 100 */ }
/* 101 */ serializefromobject_isNull10 = serializefromobject_value10 == null;
/* 102 */ }
/* 103 */
/* 104 */ if (serializefromobject_isNull10) {
/* 105 */ throw new RuntimeException(((java.lang.String) references[5]));
/* 106 */ }
/* 107 */ boolean serializefromobject_isNull8 = true;
/* 108 */ int serializefromobject_value8 = -1;
/* 109 */ if (!false) {
/* 110 */ serializefromobject_isNull8 = false;
/* 111 */ if (!serializefromobject_isNull8) {
/* 112 */ serializefromobject_value8 = serializefromobject_value10.id();
/* 113 */ }
/* 114 */
/* 115 */ }
/* 116 */ if (serializefromobject_isNull8) {
/* 117 */ serializefromobject_values[0] = null;
/* 118 */ } else {
/* 119 */ serializefromobject_values[0] = serializefromobject_value8;
/* 120 */ }
/* 121 */ final InternalRow serializefromobject_value7 = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(serializefromobject_values);
/* 122 */ this.serializefromobject_values = null;
/* 123 */ serializefromobject_isNull = false;
/* 124 */ serializefromobject_value = serializefromobject_value7;
/* 125 */ }
/* 126 */ serializefromobject_holder.reset();
/* 127 */
/* 128 */ serializefromobject_rowWriter.zeroOutNullBytes();
/* 129 */
/* 130 */ if (serializefromobject_isNull) {
/* 131 */ serializefromobject_rowWriter.setNullAt(0);
/* 132 */ } else {
/* 133 */ // Remember the current cursor so that we can calculate how many bytes are
/* 134 */ // written later.
/* 135 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 136 */
/* 137 */ if (serializefromobject_value instanceof UnsafeRow) {
/* 138 */ final int serializefromobject_sizeInBytes = ((UnsafeRow) serializefromobject_value).getSizeInBytes();
/* 139 */ // grow the global buffer before writing data.
/* 140 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 141 */ ((UnsafeRow) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 142 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 143 */
/* 144 */ } else {
/* 145 */ serializefromobject_rowWriter1.reset();
/* 146 */
/* 147 */ final int serializefromobject_fieldName = serializefromobject_value.getInt(0);
/* 148 */ if (serializefromobject_value.isNullAt(0)) {
/* 149 */ serializefromobject_rowWriter1.setNullAt(0);
/* 150 */ } else {
/* 151 */ serializefromobject_rowWriter1.write(0, serializefromobject_fieldName);
/* 152 */ }
/* 153 */ }
/* 154 */
/* 155 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 156 */ }
/* 157 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 158 */ append(serializefromobject_result);
/* 159 */ if (shouldStop()) return;
/* 160 */ }
/* 161 */ }
/* 162 */ }
{code}
> 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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org