You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Koert Kuipers <ko...@tresata.com> on 2016/05/17 22:29:52 UTC
CompileException for spark-sql generated code in 2.0.0-SNAPSHOT
hello all, we are slowly expanding our test coverage for spark
2.0.0-SNAPSHOT to more in-house projects. today i ran into this issue...
this runs fine:
val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
df
.map(row => row)(RowEncoder(df.schema))
.select("a", "b")
.show
however this fails:
val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
df
.map(row => row)(RowEncoder(df.schema))
.select("b", "a")
.show
the error is:
java.lang.Exception: failed to compile:
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line
94, Column 57: Expression "mapelements_isNull" is not an rvalue
/* 001 */ public Object generate(Object[] references) {
/* 002 */ return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * Project [b#11,a#10]
/* 007 */ +- SerializeFromObject [if (input[0,
org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class org.ap...
/* 008 */ */
/* 009 */ final class GeneratedIterator extends
org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */ private Object[] references;
/* 011 */ private scala.collection.Iterator inputadapter_input;
/* 012 */ private UnsafeRow project_result;
/* 013 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
project_holder;
/* 014 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
project_rowWriter;
/* 015 */ private Object[] deserializetoobject_values;
/* 016 */ private org.apache.spark.sql.types.StructType
deserializetoobject_schema;
/* 017 */ private UnsafeRow deserializetoobject_result;
/* 018 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
deserializetoobject_holder;
/* 019 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
deserializetoobject_rowWriter;
/* 020 */ private UnsafeRow mapelements_result;
/* 021 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
mapelements_holder;
/* 022 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
mapelements_rowWriter;
/* 023 */ private UnsafeRow serializefromobject_result;
/* 024 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
serializefromobject_holder;
/* 025 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
serializefromobject_rowWriter;
/* 026 */ private UnsafeRow project_result1;
/* 027 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
project_holder1;
/* 028 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
project_rowWriter1;
/* 029 */
/* 030 */ public GeneratedIterator(Object[] references) {
/* 031 */ this.references = references;
/* 032 */ }
/* 033 */
/* 034 */ public void init(int index, scala.collection.Iterator inputs[])
{
/* 035 */ partitionIndex = index;
/* 036 */ inputadapter_input = inputs[0];
/* 037 */ project_result = new UnsafeRow(2);
/* 038 */ this.project_holder = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result,
64);
/* 039 */ this.project_rowWriter = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder,
2);
/* 040 */
/* 041 */ this.deserializetoobject_schema =
(org.apache.spark.sql.types.StructType) references[0];
/* 042 */ deserializetoobject_result = new UnsafeRow(1);
/* 043 */ this.deserializetoobject_holder = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result,
32);
/* 044 */ this.deserializetoobject_rowWriter = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder,
1);
/* 045 */ mapelements_result = new UnsafeRow(1);
/* 046 */ this.mapelements_holder = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result,
32);
/* 047 */ this.mapelements_rowWriter = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder,
1);
/* 048 */ serializefromobject_result = new UnsafeRow(2);
/* 049 */ this.serializefromobject_holder = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result,
64);
/* 050 */ this.serializefromobject_rowWriter = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder,
2);
/* 051 */ project_result1 = new UnsafeRow(2);
/* 052 */ this.project_holder1 = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result1,
64);
/* 053 */ this.project_rowWriter1 = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder1,
2);
/* 054 */ }
/* 055 */
/* 056 */ protected void processNext() throws java.io.IOException {
/* 057 */ /*** PRODUCE: Project [b#11,a#10] */
/* 058 */
/* 059 */ /*** PRODUCE: SerializeFromObject [if (input[0,
org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
org.apache.spark.unsafe.types... */
/* 060 */
/* 061 */ /*** PRODUCE: MapElements <function1>, obj#9:
org.apache.spark.sql.Row */
/* 062 */
/* 063 */ /*** PRODUCE: DeserializeToObject createexternalrow(if
(isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
b#6.toString, StructFi... */
/* 064 */
/* 065 */ /*** PRODUCE: Project [_1#2 AS a#5,_2#3 AS b#6] */
/* 066 */
/* 067 */ /*** PRODUCE: INPUT */
/* 068 */
/* 069 */ while (inputadapter_input.hasNext()) {
/* 070 */ InternalRow inputadapter_row = (InternalRow)
inputadapter_input.next();
/* 071 */ /*** CONSUME: Project [_1#2 AS a#5,_2#3 AS b#6] */
/* 072 */
/* 073 */ /*** CONSUME: DeserializeToObject createexternalrow(if
(isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
b#6.toString, StructFi... */
/* 074 */ /* input[0, string] */
/* 075 */ /* input[0, string] */
/* 076 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 077 */ UTF8String inputadapter_value = inputadapter_isNull ? null
: (inputadapter_row.getUTF8String(0));
/* 078 */ /* input[1, string] */
/* 079 */ /* input[1, string] */
/* 080 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 081 */ UTF8String inputadapter_value1 = inputadapter_isNull1 ?
null : (inputadapter_row.getUTF8String(1));
/* 082 */
/* 083 */ /*** CONSUME: MapElements <function1>, obj#9:
org.apache.spark.sql.Row */
/* 084 */
/* 085 */ /*** CONSUME: SerializeFromObject [if (input[0,
org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
org.apache.spark.unsafe.types... */
/* 086 */
/* 087 */ /*** CONSUME: Project [b#11,a#10] */
/* 088 */
/* 089 */ /*** CONSUME: WholeStageCodegen */
/* 090 */
/* 091 */ /* input[1, string] */
/* 092 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
StringTy... */
/* 093 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
/* 094 */ boolean serializefromobject_isNull9 = mapelements_isNull ||
false;
/* 095 */ final boolean serializefromobject_value9 =
serializefromobject_isNull9 ? false : mapelements_value.isNullAt(1);
/* 096 */ boolean serializefromobject_isNull8 = false;
/* 097 */ UTF8String serializefromobject_value8 = null;
/* 098 */ if (!serializefromobject_isNull9 &&
serializefromobject_value9) {
/* 099 */ /* null */
/* 100 */ final UTF8String serializefromobject_value12 = null;
/* 101 */ serializefromobject_isNull8 = true;
/* 102 */ serializefromobject_value8 = serializefromobject_value12;
/* 103 */ } else {
/* 104 */ /* staticinvoke(class
org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
getexternalrowfield(input[0, org.apache.spa... */
/* 105 */ /* getexternalrowfield(input[0,
org.apache.spark.sql.Row], 1, ObjectType(class java.lang.String)) */
/* 106 */ if (mapelements_isNull) {
/* 107 */ throw new RuntimeException("The input external row
cannot be null.");
/* 108 */ }
/* 109 */
/* 110 */ if (mapelements_value.isNullAt(1)) {
/* 111 */ throw new RuntimeException("The 1th field of input row
cannot be null.");
/* 112 */ }
/* 113 */
/* 114 */ final java.lang.String serializefromobject_value14 =
(java.lang.String) mapelements_value.get(1);
/* 115 */ boolean serializefromobject_isNull13 = false;
/* 116 */ final UTF8String serializefromobject_value13 =
serializefromobject_isNull13 ? null :
org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value14);
/* 117 */ serializefromobject_isNull13 =
serializefromobject_value13 == null;
/* 118 */ serializefromobject_isNull8 =
serializefromobject_isNull13;
/* 119 */ serializefromobject_value8 = serializefromobject_value13;
/* 120 */ }
/* 121 */ /* input[0, string] */
/* 122 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
StringTy... */
/* 123 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
/* 124 */ /* input[0, org.apache.spark.sql.Row] */
/* 125 */ /* <function1>.apply */
/* 126 */ /* <function1> */
/* 127 */ /* expression: <function1> */
/* 128 */ Object mapelements_obj = ((Expression)
references[1]).eval(null);
/* 129 */ scala.Function1 mapelements_value1 = (scala.Function1)
mapelements_obj;
/* 130 */ /* input[0, org.apache.spark.sql.Row] */
/* 131 */ /* createexternalrow(if (isnull(input[0, string])) null
else input[0, string].toString, if (isnull(input[1, string])) null else
inp... */
/* 132 */ deserializetoobject_values = new Object[2];
/* 133 */ /* if (isnull(input[0, string])) null else input[0,
string].toString */
/* 134 */ boolean deserializetoobject_isNull1 = false;
/* 135 */ java.lang.String deserializetoobject_value1 = null;
/* 136 */ if (!false && inputadapter_isNull) {
/* 137 */ /* null */
/* 138 */ final java.lang.String deserializetoobject_value4 = null;
/* 139 */ deserializetoobject_isNull1 = true;
/* 140 */ deserializetoobject_value1 = deserializetoobject_value4;
/* 141 */ } else {
/* 142 */ /* input[0, string].toString */
/* 143 */ boolean deserializetoobject_isNull5 = inputadapter_isNull;
/* 144 */ final java.lang.String deserializetoobject_value5 =
deserializetoobject_isNull5 ? null : (java.lang.String)
inputadapter_value.toString();
/* 145 */ deserializetoobject_isNull5 = deserializetoobject_value5
== null;
/* 146 */ deserializetoobject_isNull1 = deserializetoobject_isNull5;
/* 147 */ deserializetoobject_value1 = deserializetoobject_value5;
/* 148 */ }
/* 149 */ if (deserializetoobject_isNull1) {
/* 150 */ deserializetoobject_values[0] = null;
/* 151 */ } else {
/* 152 */ deserializetoobject_values[0] =
deserializetoobject_value1;
/* 153 */ }
/* 154 */ /* if (isnull(input[1, string])) null else input[1,
string].toString */
/* 155 */ boolean deserializetoobject_isNull7 = false;
/* 156 */ java.lang.String deserializetoobject_value7 = null;
/* 157 */ if (!false && inputadapter_isNull1) {
/* 158 */ /* null */
/* 159 */ final java.lang.String deserializetoobject_value10 = null;
/* 160 */ deserializetoobject_isNull7 = true;
/* 161 */ deserializetoobject_value7 = deserializetoobject_value10;
/* 162 */ } else {
/* 163 */ /* input[1, string].toString */
/* 164 */ boolean deserializetoobject_isNull11 =
inputadapter_isNull1;
/* 165 */ final java.lang.String deserializetoobject_value11 =
deserializetoobject_isNull11 ? null : (java.lang.String)
inputadapter_value1.toString();
/* 166 */ deserializetoobject_isNull11 =
deserializetoobject_value11 == null;
/* 167 */ deserializetoobject_isNull7 =
deserializetoobject_isNull11;
/* 168 */ deserializetoobject_value7 = deserializetoobject_value11;
/* 169 */ }
/* 170 */ if (deserializetoobject_isNull7) {
/* 171 */ deserializetoobject_values[1] = null;
/* 172 */ } else {
/* 173 */ deserializetoobject_values[1] =
deserializetoobject_value7;
/* 174 */ }
/* 175 */
/* 176 */ final org.apache.spark.sql.Row deserializetoobject_value =
new
org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(deserializetoobject_values,
this.deserializetoobject_schema);
/* 177 */ boolean mapelements_isNull = false || false;
/* 178 */ final org.apache.spark.sql.Row mapelements_value =
mapelements_isNull ? null : (org.apache.spark.sql.Row)
mapelements_value1.apply(deserializetoobject_value);
/* 179 */ mapelements_isNull = mapelements_value == null;
/* 180 */
/* 181 */ boolean serializefromobject_isNull1 = mapelements_isNull ||
false;
/* 182 */ final boolean serializefromobject_value1 =
serializefromobject_isNull1 ? false : mapelements_value.isNullAt(0);
/* 183 */ boolean serializefromobject_isNull = false;
/* 184 */ UTF8String serializefromobject_value = null;
/* 185 */ if (!serializefromobject_isNull1 &&
serializefromobject_value1) {
/* 186 */ /* null */
/* 187 */ final UTF8String serializefromobject_value4 = null;
/* 188 */ serializefromobject_isNull = true;
/* 189 */ serializefromobject_value = serializefromobject_value4;
/* 190 */ } else {
/* 191 */ /* staticinvoke(class
org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
getexternalrowfield(input[0, org.apache.spa... */
/* 192 */ /* getexternalrowfield(input[0,
org.apache.spark.sql.Row], 0, ObjectType(class java.lang.String)) */
/* 193 */ if (mapelements_isNull) {
/* 194 */ throw new RuntimeException("The input external row
cannot be null.");
/* 195 */ }
/* 196 */
/* 197 */ if (mapelements_value.isNullAt(0)) {
/* 198 */ throw new RuntimeException("The 0th field of input row
cannot be null.");
/* 199 */ }
/* 200 */
/* 201 */ final java.lang.String serializefromobject_value6 =
(java.lang.String) mapelements_value.get(0);
/* 202 */ boolean serializefromobject_isNull5 = false;
/* 203 */ final UTF8String serializefromobject_value5 =
serializefromobject_isNull5 ? null :
org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value6);
/* 204 */ serializefromobject_isNull5 = serializefromobject_value5
== null;
/* 205 */ serializefromobject_isNull = serializefromobject_isNull5;
/* 206 */ serializefromobject_value = serializefromobject_value5;
/* 207 */ }
/* 208 */ project_holder1.reset();
/* 209 */
/* 210 */ project_rowWriter1.zeroOutNullBytes();
/* 211 */
/* 212 */ if (serializefromobject_isNull8) {
/* 213 */ project_rowWriter1.setNullAt(0);
/* 214 */ } else {
/* 215 */ project_rowWriter1.write(0, serializefromobject_value8);
/* 216 */ }
/* 217 */
/* 218 */ if (serializefromobject_isNull) {
/* 219 */ project_rowWriter1.setNullAt(1);
/* 220 */ } else {
/* 221 */ project_rowWriter1.write(1, serializefromobject_value);
/* 222 */ }
/* 223 */ project_result1.setTotalSize(project_holder1.totalSize());
/* 224 */ append(project_result1);
/* 225 */ if (shouldStop()) return;
/* 226 */ }
/* 227 */ }
/* 228 */ }
Re: CompileException for spark-sql generated code in 2.0.0-SNAPSHOT
Posted by Koert Kuipers <ko...@tresata.com>.
https://issues.apache.org/jira/browse/SPARK-15384
On Tue, May 17, 2016 at 9:06 PM, Michael Armbrust <mi...@databricks.com>
wrote:
> Yeah, can you open a JIRA with that reproduction please? You can ping me
> on it.
>
> On Tue, May 17, 2016 at 4:55 PM, Reynold Xin <rx...@databricks.com> wrote:
>
>> It seems like the problem here is that we are not using unique names
>> for mapelements_isNull?
>>
>>
>>
>> On Tue, May 17, 2016 at 3:29 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>
>>> hello all, we are slowly expanding our test coverage for spark
>>> 2.0.0-SNAPSHOT to more in-house projects. today i ran into this issue...
>>>
>>> this runs fine:
>>> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
>>> df
>>> .map(row => row)(RowEncoder(df.schema))
>>> .select("a", "b")
>>> .show
>>>
>>> however this fails:
>>> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
>>> df
>>> .map(row => row)(RowEncoder(df.schema))
>>> .select("b", "a")
>>> .show
>>>
>>> the error is:
>>> java.lang.Exception: failed to compile:
>>> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line
>>> 94, Column 57: Expression "mapelements_isNull" is not an rvalue
>>> /* 001 */ public Object generate(Object[] references) {
>>> /* 002 */ return new GeneratedIterator(references);
>>> /* 003 */ }
>>> /* 004 */
>>> /* 005 */ /** Codegened pipeline for:
>>> /* 006 */ * Project [b#11,a#10]
>>> /* 007 */ +- SerializeFromObject [if (input[0,
>>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class org.ap...
>>> /* 008 */ */
>>> /* 009 */ final class GeneratedIterator extends
>>> org.apache.spark.sql.execution.BufferedRowIterator {
>>> /* 010 */ private Object[] references;
>>> /* 011 */ private scala.collection.Iterator inputadapter_input;
>>> /* 012 */ private UnsafeRow project_result;
>>> /* 013 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>>> project_holder;
>>> /* 014 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>>> project_rowWriter;
>>> /* 015 */ private Object[] deserializetoobject_values;
>>> /* 016 */ private org.apache.spark.sql.types.StructType
>>> deserializetoobject_schema;
>>> /* 017 */ private UnsafeRow deserializetoobject_result;
>>> /* 018 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>>> deserializetoobject_holder;
>>> /* 019 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>>> deserializetoobject_rowWriter;
>>> /* 020 */ private UnsafeRow mapelements_result;
>>> /* 021 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>>> mapelements_holder;
>>> /* 022 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>>> mapelements_rowWriter;
>>> /* 023 */ private UnsafeRow serializefromobject_result;
>>> /* 024 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>>> serializefromobject_holder;
>>> /* 025 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>>> serializefromobject_rowWriter;
>>> /* 026 */ private UnsafeRow project_result1;
>>> /* 027 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>>> project_holder1;
>>> /* 028 */ private
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>>> project_rowWriter1;
>>> /* 029 */
>>> /* 030 */ public GeneratedIterator(Object[] references) {
>>> /* 031 */ this.references = references;
>>> /* 032 */ }
>>> /* 033 */
>>> /* 034 */ public void init(int index, scala.collection.Iterator
>>> inputs[]) {
>>> /* 035 */ partitionIndex = index;
>>> /* 036 */ inputadapter_input = inputs[0];
>>> /* 037 */ project_result = new UnsafeRow(2);
>>> /* 038 */ this.project_holder = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result,
>>> 64);
>>> /* 039 */ this.project_rowWriter = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder,
>>> 2);
>>> /* 040 */
>>> /* 041 */ this.deserializetoobject_schema =
>>> (org.apache.spark.sql.types.StructType) references[0];
>>> /* 042 */ deserializetoobject_result = new UnsafeRow(1);
>>> /* 043 */ this.deserializetoobject_holder = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result,
>>> 32);
>>> /* 044 */ this.deserializetoobject_rowWriter = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder,
>>> 1);
>>> /* 045 */ mapelements_result = new UnsafeRow(1);
>>> /* 046 */ this.mapelements_holder = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result,
>>> 32);
>>> /* 047 */ this.mapelements_rowWriter = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder,
>>> 1);
>>> /* 048 */ serializefromobject_result = new UnsafeRow(2);
>>> /* 049 */ this.serializefromobject_holder = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result,
>>> 64);
>>> /* 050 */ this.serializefromobject_rowWriter = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder,
>>> 2);
>>> /* 051 */ project_result1 = new UnsafeRow(2);
>>> /* 052 */ this.project_holder1 = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result1,
>>> 64);
>>> /* 053 */ this.project_rowWriter1 = new
>>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder1,
>>> 2);
>>> /* 054 */ }
>>> /* 055 */
>>> /* 056 */ protected void processNext() throws java.io.IOException {
>>> /* 057 */ /*** PRODUCE: Project [b#11,a#10] */
>>> /* 058 */
>>> /* 059 */ /*** PRODUCE: SerializeFromObject [if (input[0,
>>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
>>> org.apache.spark.unsafe.types... */
>>> /* 060 */
>>> /* 061 */ /*** PRODUCE: MapElements <function1>, obj#9:
>>> org.apache.spark.sql.Row */
>>> /* 062 */
>>> /* 063 */ /*** PRODUCE: DeserializeToObject createexternalrow(if
>>> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
>>> b#6.toString, StructFi... */
>>> /* 064 */
>>> /* 065 */ /*** PRODUCE: Project [_1#2 AS a#5,_2#3 AS b#6] */
>>> /* 066 */
>>> /* 067 */ /*** PRODUCE: INPUT */
>>> /* 068 */
>>> /* 069 */ while (inputadapter_input.hasNext()) {
>>> /* 070 */ InternalRow inputadapter_row = (InternalRow)
>>> inputadapter_input.next();
>>> /* 071 */ /*** CONSUME: Project [_1#2 AS a#5,_2#3 AS b#6] */
>>> /* 072 */
>>> /* 073 */ /*** CONSUME: DeserializeToObject createexternalrow(if
>>> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
>>> b#6.toString, StructFi... */
>>> /* 074 */ /* input[0, string] */
>>> /* 075 */ /* input[0, string] */
>>> /* 076 */ boolean inputadapter_isNull =
>>> inputadapter_row.isNullAt(0);
>>> /* 077 */ UTF8String inputadapter_value = inputadapter_isNull ?
>>> null : (inputadapter_row.getUTF8String(0));
>>> /* 078 */ /* input[1, string] */
>>> /* 079 */ /* input[1, string] */
>>> /* 080 */ boolean inputadapter_isNull1 =
>>> inputadapter_row.isNullAt(1);
>>> /* 081 */ UTF8String inputadapter_value1 = inputadapter_isNull1 ?
>>> null : (inputadapter_row.getUTF8String(1));
>>> /* 082 */
>>> /* 083 */ /*** CONSUME: MapElements <function1>, obj#9:
>>> org.apache.spark.sql.Row */
>>> /* 084 */
>>> /* 085 */ /*** CONSUME: SerializeFromObject [if (input[0,
>>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
>>> org.apache.spark.unsafe.types... */
>>> /* 086 */
>>> /* 087 */ /*** CONSUME: Project [b#11,a#10] */
>>> /* 088 */
>>> /* 089 */ /*** CONSUME: WholeStageCodegen */
>>> /* 090 */
>>> /* 091 */ /* input[1, string] */
>>> /* 092 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
>>> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
>>> StringTy... */
>>> /* 093 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
>>> /* 094 */ boolean serializefromobject_isNull9 = mapelements_isNull
>>> || false;
>>> /* 095 */ final boolean serializefromobject_value9 =
>>> serializefromobject_isNull9 ? false : mapelements_value.isNullAt(1);
>>> /* 096 */ boolean serializefromobject_isNull8 = false;
>>> /* 097 */ UTF8String serializefromobject_value8 = null;
>>> /* 098 */ if (!serializefromobject_isNull9 &&
>>> serializefromobject_value9) {
>>> /* 099 */ /* null */
>>> /* 100 */ final UTF8String serializefromobject_value12 = null;
>>> /* 101 */ serializefromobject_isNull8 = true;
>>> /* 102 */ serializefromobject_value8 =
>>> serializefromobject_value12;
>>> /* 103 */ } else {
>>> /* 104 */ /* staticinvoke(class
>>> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
>>> getexternalrowfield(input[0, org.apache.spa... */
>>> /* 105 */ /* getexternalrowfield(input[0,
>>> org.apache.spark.sql.Row], 1, ObjectType(class java.lang.String)) */
>>> /* 106 */ if (mapelements_isNull) {
>>> /* 107 */ throw new RuntimeException("The input external row
>>> cannot be null.");
>>> /* 108 */ }
>>> /* 109 */
>>> /* 110 */ if (mapelements_value.isNullAt(1)) {
>>> /* 111 */ throw new RuntimeException("The 1th field of input
>>> row cannot be null.");
>>> /* 112 */ }
>>> /* 113 */
>>> /* 114 */ final java.lang.String serializefromobject_value14 =
>>> (java.lang.String) mapelements_value.get(1);
>>> /* 115 */ boolean serializefromobject_isNull13 = false;
>>> /* 116 */ final UTF8String serializefromobject_value13 =
>>> serializefromobject_isNull13 ? null :
>>> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value14);
>>> /* 117 */ serializefromobject_isNull13 =
>>> serializefromobject_value13 == null;
>>> /* 118 */ serializefromobject_isNull8 =
>>> serializefromobject_isNull13;
>>> /* 119 */ serializefromobject_value8 =
>>> serializefromobject_value13;
>>> /* 120 */ }
>>> /* 121 */ /* input[0, string] */
>>> /* 122 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
>>> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
>>> StringTy... */
>>> /* 123 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
>>> /* 124 */ /* input[0, org.apache.spark.sql.Row] */
>>> /* 125 */ /* <function1>.apply */
>>> /* 126 */ /* <function1> */
>>> /* 127 */ /* expression: <function1> */
>>> /* 128 */ Object mapelements_obj = ((Expression)
>>> references[1]).eval(null);
>>> /* 129 */ scala.Function1 mapelements_value1 = (scala.Function1)
>>> mapelements_obj;
>>> /* 130 */ /* input[0, org.apache.spark.sql.Row] */
>>> /* 131 */ /* createexternalrow(if (isnull(input[0, string])) null
>>> else input[0, string].toString, if (isnull(input[1, string])) null else
>>> inp... */
>>> /* 132 */ deserializetoobject_values = new Object[2];
>>> /* 133 */ /* if (isnull(input[0, string])) null else input[0,
>>> string].toString */
>>> /* 134 */ boolean deserializetoobject_isNull1 = false;
>>> /* 135 */ java.lang.String deserializetoobject_value1 = null;
>>> /* 136 */ if (!false && inputadapter_isNull) {
>>> /* 137 */ /* null */
>>> /* 138 */ final java.lang.String deserializetoobject_value4 =
>>> null;
>>> /* 139 */ deserializetoobject_isNull1 = true;
>>> /* 140 */ deserializetoobject_value1 =
>>> deserializetoobject_value4;
>>> /* 141 */ } else {
>>> /* 142 */ /* input[0, string].toString */
>>> /* 143 */ boolean deserializetoobject_isNull5 =
>>> inputadapter_isNull;
>>> /* 144 */ final java.lang.String deserializetoobject_value5 =
>>> deserializetoobject_isNull5 ? null : (java.lang.String)
>>> inputadapter_value.toString();
>>> /* 145 */ deserializetoobject_isNull5 =
>>> deserializetoobject_value5 == null;
>>> /* 146 */ deserializetoobject_isNull1 =
>>> deserializetoobject_isNull5;
>>> /* 147 */ deserializetoobject_value1 =
>>> deserializetoobject_value5;
>>> /* 148 */ }
>>> /* 149 */ if (deserializetoobject_isNull1) {
>>> /* 150 */ deserializetoobject_values[0] = null;
>>> /* 151 */ } else {
>>> /* 152 */ deserializetoobject_values[0] =
>>> deserializetoobject_value1;
>>> /* 153 */ }
>>> /* 154 */ /* if (isnull(input[1, string])) null else input[1,
>>> string].toString */
>>> /* 155 */ boolean deserializetoobject_isNull7 = false;
>>> /* 156 */ java.lang.String deserializetoobject_value7 = null;
>>> /* 157 */ if (!false && inputadapter_isNull1) {
>>> /* 158 */ /* null */
>>> /* 159 */ final java.lang.String deserializetoobject_value10 =
>>> null;
>>> /* 160 */ deserializetoobject_isNull7 = true;
>>> /* 161 */ deserializetoobject_value7 =
>>> deserializetoobject_value10;
>>> /* 162 */ } else {
>>> /* 163 */ /* input[1, string].toString */
>>> /* 164 */ boolean deserializetoobject_isNull11 =
>>> inputadapter_isNull1;
>>> /* 165 */ final java.lang.String deserializetoobject_value11 =
>>> deserializetoobject_isNull11 ? null : (java.lang.String)
>>> inputadapter_value1.toString();
>>> /* 166 */ deserializetoobject_isNull11 =
>>> deserializetoobject_value11 == null;
>>> /* 167 */ deserializetoobject_isNull7 =
>>> deserializetoobject_isNull11;
>>> /* 168 */ deserializetoobject_value7 =
>>> deserializetoobject_value11;
>>> /* 169 */ }
>>> /* 170 */ if (deserializetoobject_isNull7) {
>>> /* 171 */ deserializetoobject_values[1] = null;
>>> /* 172 */ } else {
>>> /* 173 */ deserializetoobject_values[1] =
>>> deserializetoobject_value7;
>>> /* 174 */ }
>>> /* 175 */
>>> /* 176 */ final org.apache.spark.sql.Row deserializetoobject_value
>>> = new
>>> org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(deserializetoobject_values,
>>> this.deserializetoobject_schema);
>>> /* 177 */ boolean mapelements_isNull = false || false;
>>> /* 178 */ final org.apache.spark.sql.Row mapelements_value =
>>> mapelements_isNull ? null : (org.apache.spark.sql.Row)
>>> mapelements_value1.apply(deserializetoobject_value);
>>> /* 179 */ mapelements_isNull = mapelements_value == null;
>>> /* 180 */
>>> /* 181 */ boolean serializefromobject_isNull1 = mapelements_isNull
>>> || false;
>>> /* 182 */ final boolean serializefromobject_value1 =
>>> serializefromobject_isNull1 ? false : mapelements_value.isNullAt(0);
>>> /* 183 */ boolean serializefromobject_isNull = false;
>>> /* 184 */ UTF8String serializefromobject_value = null;
>>> /* 185 */ if (!serializefromobject_isNull1 &&
>>> serializefromobject_value1) {
>>> /* 186 */ /* null */
>>> /* 187 */ final UTF8String serializefromobject_value4 = null;
>>> /* 188 */ serializefromobject_isNull = true;
>>> /* 189 */ serializefromobject_value = serializefromobject_value4;
>>> /* 190 */ } else {
>>> /* 191 */ /* staticinvoke(class
>>> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
>>> getexternalrowfield(input[0, org.apache.spa... */
>>> /* 192 */ /* getexternalrowfield(input[0,
>>> org.apache.spark.sql.Row], 0, ObjectType(class java.lang.String)) */
>>> /* 193 */ if (mapelements_isNull) {
>>> /* 194 */ throw new RuntimeException("The input external row
>>> cannot be null.");
>>> /* 195 */ }
>>> /* 196 */
>>> /* 197 */ if (mapelements_value.isNullAt(0)) {
>>> /* 198 */ throw new RuntimeException("The 0th field of input
>>> row cannot be null.");
>>> /* 199 */ }
>>> /* 200 */
>>> /* 201 */ final java.lang.String serializefromobject_value6 =
>>> (java.lang.String) mapelements_value.get(0);
>>> /* 202 */ boolean serializefromobject_isNull5 = false;
>>> /* 203 */ final UTF8String serializefromobject_value5 =
>>> serializefromobject_isNull5 ? null :
>>> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value6);
>>> /* 204 */ serializefromobject_isNull5 =
>>> serializefromobject_value5 == null;
>>> /* 205 */ serializefromobject_isNull =
>>> serializefromobject_isNull5;
>>> /* 206 */ serializefromobject_value = serializefromobject_value5;
>>> /* 207 */ }
>>> /* 208 */ project_holder1.reset();
>>> /* 209 */
>>> /* 210 */ project_rowWriter1.zeroOutNullBytes();
>>> /* 211 */
>>> /* 212 */ if (serializefromobject_isNull8) {
>>> /* 213 */ project_rowWriter1.setNullAt(0);
>>> /* 214 */ } else {
>>> /* 215 */ project_rowWriter1.write(0,
>>> serializefromobject_value8);
>>> /* 216 */ }
>>> /* 217 */
>>> /* 218 */ if (serializefromobject_isNull) {
>>> /* 219 */ project_rowWriter1.setNullAt(1);
>>> /* 220 */ } else {
>>> /* 221 */ project_rowWriter1.write(1, serializefromobject_value);
>>> /* 222 */ }
>>> /* 223 */
>>> project_result1.setTotalSize(project_holder1.totalSize());
>>> /* 224 */ append(project_result1);
>>> /* 225 */ if (shouldStop()) return;
>>> /* 226 */ }
>>> /* 227 */ }
>>> /* 228 */ }
>>>
>>
>>
>
Re: CompileException for spark-sql generated code in 2.0.0-SNAPSHOT
Posted by Michael Armbrust <mi...@databricks.com>.
Yeah, can you open a JIRA with that reproduction please? You can ping me
on it.
On Tue, May 17, 2016 at 4:55 PM, Reynold Xin <rx...@databricks.com> wrote:
> It seems like the problem here is that we are not using unique names
> for mapelements_isNull?
>
>
>
> On Tue, May 17, 2016 at 3:29 PM, Koert Kuipers <ko...@tresata.com> wrote:
>
>> hello all, we are slowly expanding our test coverage for spark
>> 2.0.0-SNAPSHOT to more in-house projects. today i ran into this issue...
>>
>> this runs fine:
>> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
>> df
>> .map(row => row)(RowEncoder(df.schema))
>> .select("a", "b")
>> .show
>>
>> however this fails:
>> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
>> df
>> .map(row => row)(RowEncoder(df.schema))
>> .select("b", "a")
>> .show
>>
>> the error is:
>> java.lang.Exception: failed to compile:
>> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line
>> 94, Column 57: Expression "mapelements_isNull" is not an rvalue
>> /* 001 */ public Object generate(Object[] references) {
>> /* 002 */ return new GeneratedIterator(references);
>> /* 003 */ }
>> /* 004 */
>> /* 005 */ /** Codegened pipeline for:
>> /* 006 */ * Project [b#11,a#10]
>> /* 007 */ +- SerializeFromObject [if (input[0,
>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class org.ap...
>> /* 008 */ */
>> /* 009 */ final class GeneratedIterator extends
>> org.apache.spark.sql.execution.BufferedRowIterator {
>> /* 010 */ private Object[] references;
>> /* 011 */ private scala.collection.Iterator inputadapter_input;
>> /* 012 */ private UnsafeRow project_result;
>> /* 013 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>> project_holder;
>> /* 014 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>> project_rowWriter;
>> /* 015 */ private Object[] deserializetoobject_values;
>> /* 016 */ private org.apache.spark.sql.types.StructType
>> deserializetoobject_schema;
>> /* 017 */ private UnsafeRow deserializetoobject_result;
>> /* 018 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>> deserializetoobject_holder;
>> /* 019 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>> deserializetoobject_rowWriter;
>> /* 020 */ private UnsafeRow mapelements_result;
>> /* 021 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>> mapelements_holder;
>> /* 022 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>> mapelements_rowWriter;
>> /* 023 */ private UnsafeRow serializefromobject_result;
>> /* 024 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>> serializefromobject_holder;
>> /* 025 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>> serializefromobject_rowWriter;
>> /* 026 */ private UnsafeRow project_result1;
>> /* 027 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
>> project_holder1;
>> /* 028 */ private
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
>> project_rowWriter1;
>> /* 029 */
>> /* 030 */ public GeneratedIterator(Object[] references) {
>> /* 031 */ this.references = references;
>> /* 032 */ }
>> /* 033 */
>> /* 034 */ public void init(int index, scala.collection.Iterator
>> inputs[]) {
>> /* 035 */ partitionIndex = index;
>> /* 036 */ inputadapter_input = inputs[0];
>> /* 037 */ project_result = new UnsafeRow(2);
>> /* 038 */ this.project_holder = new
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result,
>> 64);
>> /* 039 */ this.project_rowWriter = new
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder,
>> 2);
>> /* 040 */
>> /* 041 */ this.deserializetoobject_schema =
>> (org.apache.spark.sql.types.StructType) references[0];
>> /* 042 */ deserializetoobject_result = new UnsafeRow(1);
>> /* 043 */ this.deserializetoobject_holder = new
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result,
>> 32);
>> /* 044 */ this.deserializetoobject_rowWriter = new
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder,
>> 1);
>> /* 045 */ mapelements_result = new UnsafeRow(1);
>> /* 046 */ this.mapelements_holder = new
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result,
>> 32);
>> /* 047 */ this.mapelements_rowWriter = new
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder,
>> 1);
>> /* 048 */ serializefromobject_result = new UnsafeRow(2);
>> /* 049 */ this.serializefromobject_holder = new
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result,
>> 64);
>> /* 050 */ this.serializefromobject_rowWriter = new
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder,
>> 2);
>> /* 051 */ project_result1 = new UnsafeRow(2);
>> /* 052 */ this.project_holder1 = new
>> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result1,
>> 64);
>> /* 053 */ this.project_rowWriter1 = new
>> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder1,
>> 2);
>> /* 054 */ }
>> /* 055 */
>> /* 056 */ protected void processNext() throws java.io.IOException {
>> /* 057 */ /*** PRODUCE: Project [b#11,a#10] */
>> /* 058 */
>> /* 059 */ /*** PRODUCE: SerializeFromObject [if (input[0,
>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
>> org.apache.spark.unsafe.types... */
>> /* 060 */
>> /* 061 */ /*** PRODUCE: MapElements <function1>, obj#9:
>> org.apache.spark.sql.Row */
>> /* 062 */
>> /* 063 */ /*** PRODUCE: DeserializeToObject createexternalrow(if
>> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
>> b#6.toString, StructFi... */
>> /* 064 */
>> /* 065 */ /*** PRODUCE: Project [_1#2 AS a#5,_2#3 AS b#6] */
>> /* 066 */
>> /* 067 */ /*** PRODUCE: INPUT */
>> /* 068 */
>> /* 069 */ while (inputadapter_input.hasNext()) {
>> /* 070 */ InternalRow inputadapter_row = (InternalRow)
>> inputadapter_input.next();
>> /* 071 */ /*** CONSUME: Project [_1#2 AS a#5,_2#3 AS b#6] */
>> /* 072 */
>> /* 073 */ /*** CONSUME: DeserializeToObject createexternalrow(if
>> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
>> b#6.toString, StructFi... */
>> /* 074 */ /* input[0, string] */
>> /* 075 */ /* input[0, string] */
>> /* 076 */ boolean inputadapter_isNull =
>> inputadapter_row.isNullAt(0);
>> /* 077 */ UTF8String inputadapter_value = inputadapter_isNull ?
>> null : (inputadapter_row.getUTF8String(0));
>> /* 078 */ /* input[1, string] */
>> /* 079 */ /* input[1, string] */
>> /* 080 */ boolean inputadapter_isNull1 =
>> inputadapter_row.isNullAt(1);
>> /* 081 */ UTF8String inputadapter_value1 = inputadapter_isNull1 ?
>> null : (inputadapter_row.getUTF8String(1));
>> /* 082 */
>> /* 083 */ /*** CONSUME: MapElements <function1>, obj#9:
>> org.apache.spark.sql.Row */
>> /* 084 */
>> /* 085 */ /*** CONSUME: SerializeFromObject [if (input[0,
>> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
>> org.apache.spark.unsafe.types... */
>> /* 086 */
>> /* 087 */ /*** CONSUME: Project [b#11,a#10] */
>> /* 088 */
>> /* 089 */ /*** CONSUME: WholeStageCodegen */
>> /* 090 */
>> /* 091 */ /* input[1, string] */
>> /* 092 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
>> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
>> StringTy... */
>> /* 093 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
>> /* 094 */ boolean serializefromobject_isNull9 = mapelements_isNull
>> || false;
>> /* 095 */ final boolean serializefromobject_value9 =
>> serializefromobject_isNull9 ? false : mapelements_value.isNullAt(1);
>> /* 096 */ boolean serializefromobject_isNull8 = false;
>> /* 097 */ UTF8String serializefromobject_value8 = null;
>> /* 098 */ if (!serializefromobject_isNull9 &&
>> serializefromobject_value9) {
>> /* 099 */ /* null */
>> /* 100 */ final UTF8String serializefromobject_value12 = null;
>> /* 101 */ serializefromobject_isNull8 = true;
>> /* 102 */ serializefromobject_value8 =
>> serializefromobject_value12;
>> /* 103 */ } else {
>> /* 104 */ /* staticinvoke(class
>> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
>> getexternalrowfield(input[0, org.apache.spa... */
>> /* 105 */ /* getexternalrowfield(input[0,
>> org.apache.spark.sql.Row], 1, ObjectType(class java.lang.String)) */
>> /* 106 */ if (mapelements_isNull) {
>> /* 107 */ throw new RuntimeException("The input external row
>> cannot be null.");
>> /* 108 */ }
>> /* 109 */
>> /* 110 */ if (mapelements_value.isNullAt(1)) {
>> /* 111 */ throw new RuntimeException("The 1th field of input
>> row cannot be null.");
>> /* 112 */ }
>> /* 113 */
>> /* 114 */ final java.lang.String serializefromobject_value14 =
>> (java.lang.String) mapelements_value.get(1);
>> /* 115 */ boolean serializefromobject_isNull13 = false;
>> /* 116 */ final UTF8String serializefromobject_value13 =
>> serializefromobject_isNull13 ? null :
>> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value14);
>> /* 117 */ serializefromobject_isNull13 =
>> serializefromobject_value13 == null;
>> /* 118 */ serializefromobject_isNull8 =
>> serializefromobject_isNull13;
>> /* 119 */ serializefromobject_value8 =
>> serializefromobject_value13;
>> /* 120 */ }
>> /* 121 */ /* input[0, string] */
>> /* 122 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
>> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
>> StringTy... */
>> /* 123 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
>> /* 124 */ /* input[0, org.apache.spark.sql.Row] */
>> /* 125 */ /* <function1>.apply */
>> /* 126 */ /* <function1> */
>> /* 127 */ /* expression: <function1> */
>> /* 128 */ Object mapelements_obj = ((Expression)
>> references[1]).eval(null);
>> /* 129 */ scala.Function1 mapelements_value1 = (scala.Function1)
>> mapelements_obj;
>> /* 130 */ /* input[0, org.apache.spark.sql.Row] */
>> /* 131 */ /* createexternalrow(if (isnull(input[0, string])) null
>> else input[0, string].toString, if (isnull(input[1, string])) null else
>> inp... */
>> /* 132 */ deserializetoobject_values = new Object[2];
>> /* 133 */ /* if (isnull(input[0, string])) null else input[0,
>> string].toString */
>> /* 134 */ boolean deserializetoobject_isNull1 = false;
>> /* 135 */ java.lang.String deserializetoobject_value1 = null;
>> /* 136 */ if (!false && inputadapter_isNull) {
>> /* 137 */ /* null */
>> /* 138 */ final java.lang.String deserializetoobject_value4 =
>> null;
>> /* 139 */ deserializetoobject_isNull1 = true;
>> /* 140 */ deserializetoobject_value1 = deserializetoobject_value4;
>> /* 141 */ } else {
>> /* 142 */ /* input[0, string].toString */
>> /* 143 */ boolean deserializetoobject_isNull5 =
>> inputadapter_isNull;
>> /* 144 */ final java.lang.String deserializetoobject_value5 =
>> deserializetoobject_isNull5 ? null : (java.lang.String)
>> inputadapter_value.toString();
>> /* 145 */ deserializetoobject_isNull5 =
>> deserializetoobject_value5 == null;
>> /* 146 */ deserializetoobject_isNull1 =
>> deserializetoobject_isNull5;
>> /* 147 */ deserializetoobject_value1 = deserializetoobject_value5;
>> /* 148 */ }
>> /* 149 */ if (deserializetoobject_isNull1) {
>> /* 150 */ deserializetoobject_values[0] = null;
>> /* 151 */ } else {
>> /* 152 */ deserializetoobject_values[0] =
>> deserializetoobject_value1;
>> /* 153 */ }
>> /* 154 */ /* if (isnull(input[1, string])) null else input[1,
>> string].toString */
>> /* 155 */ boolean deserializetoobject_isNull7 = false;
>> /* 156 */ java.lang.String deserializetoobject_value7 = null;
>> /* 157 */ if (!false && inputadapter_isNull1) {
>> /* 158 */ /* null */
>> /* 159 */ final java.lang.String deserializetoobject_value10 =
>> null;
>> /* 160 */ deserializetoobject_isNull7 = true;
>> /* 161 */ deserializetoobject_value7 =
>> deserializetoobject_value10;
>> /* 162 */ } else {
>> /* 163 */ /* input[1, string].toString */
>> /* 164 */ boolean deserializetoobject_isNull11 =
>> inputadapter_isNull1;
>> /* 165 */ final java.lang.String deserializetoobject_value11 =
>> deserializetoobject_isNull11 ? null : (java.lang.String)
>> inputadapter_value1.toString();
>> /* 166 */ deserializetoobject_isNull11 =
>> deserializetoobject_value11 == null;
>> /* 167 */ deserializetoobject_isNull7 =
>> deserializetoobject_isNull11;
>> /* 168 */ deserializetoobject_value7 =
>> deserializetoobject_value11;
>> /* 169 */ }
>> /* 170 */ if (deserializetoobject_isNull7) {
>> /* 171 */ deserializetoobject_values[1] = null;
>> /* 172 */ } else {
>> /* 173 */ deserializetoobject_values[1] =
>> deserializetoobject_value7;
>> /* 174 */ }
>> /* 175 */
>> /* 176 */ final org.apache.spark.sql.Row deserializetoobject_value
>> = new
>> org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(deserializetoobject_values,
>> this.deserializetoobject_schema);
>> /* 177 */ boolean mapelements_isNull = false || false;
>> /* 178 */ final org.apache.spark.sql.Row mapelements_value =
>> mapelements_isNull ? null : (org.apache.spark.sql.Row)
>> mapelements_value1.apply(deserializetoobject_value);
>> /* 179 */ mapelements_isNull = mapelements_value == null;
>> /* 180 */
>> /* 181 */ boolean serializefromobject_isNull1 = mapelements_isNull
>> || false;
>> /* 182 */ final boolean serializefromobject_value1 =
>> serializefromobject_isNull1 ? false : mapelements_value.isNullAt(0);
>> /* 183 */ boolean serializefromobject_isNull = false;
>> /* 184 */ UTF8String serializefromobject_value = null;
>> /* 185 */ if (!serializefromobject_isNull1 &&
>> serializefromobject_value1) {
>> /* 186 */ /* null */
>> /* 187 */ final UTF8String serializefromobject_value4 = null;
>> /* 188 */ serializefromobject_isNull = true;
>> /* 189 */ serializefromobject_value = serializefromobject_value4;
>> /* 190 */ } else {
>> /* 191 */ /* staticinvoke(class
>> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
>> getexternalrowfield(input[0, org.apache.spa... */
>> /* 192 */ /* getexternalrowfield(input[0,
>> org.apache.spark.sql.Row], 0, ObjectType(class java.lang.String)) */
>> /* 193 */ if (mapelements_isNull) {
>> /* 194 */ throw new RuntimeException("The input external row
>> cannot be null.");
>> /* 195 */ }
>> /* 196 */
>> /* 197 */ if (mapelements_value.isNullAt(0)) {
>> /* 198 */ throw new RuntimeException("The 0th field of input
>> row cannot be null.");
>> /* 199 */ }
>> /* 200 */
>> /* 201 */ final java.lang.String serializefromobject_value6 =
>> (java.lang.String) mapelements_value.get(0);
>> /* 202 */ boolean serializefromobject_isNull5 = false;
>> /* 203 */ final UTF8String serializefromobject_value5 =
>> serializefromobject_isNull5 ? null :
>> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value6);
>> /* 204 */ serializefromobject_isNull5 =
>> serializefromobject_value5 == null;
>> /* 205 */ serializefromobject_isNull =
>> serializefromobject_isNull5;
>> /* 206 */ serializefromobject_value = serializefromobject_value5;
>> /* 207 */ }
>> /* 208 */ project_holder1.reset();
>> /* 209 */
>> /* 210 */ project_rowWriter1.zeroOutNullBytes();
>> /* 211 */
>> /* 212 */ if (serializefromobject_isNull8) {
>> /* 213 */ project_rowWriter1.setNullAt(0);
>> /* 214 */ } else {
>> /* 215 */ project_rowWriter1.write(0, serializefromobject_value8);
>> /* 216 */ }
>> /* 217 */
>> /* 218 */ if (serializefromobject_isNull) {
>> /* 219 */ project_rowWriter1.setNullAt(1);
>> /* 220 */ } else {
>> /* 221 */ project_rowWriter1.write(1, serializefromobject_value);
>> /* 222 */ }
>> /* 223 */ project_result1.setTotalSize(project_holder1.totalSize());
>> /* 224 */ append(project_result1);
>> /* 225 */ if (shouldStop()) return;
>> /* 226 */ }
>> /* 227 */ }
>> /* 228 */ }
>>
>
>
Re: CompileException for spark-sql generated code in 2.0.0-SNAPSHOT
Posted by Reynold Xin <rx...@databricks.com>.
It seems like the problem here is that we are not using unique names
for mapelements_isNull?
On Tue, May 17, 2016 at 3:29 PM, Koert Kuipers <ko...@tresata.com> wrote:
> hello all, we are slowly expanding our test coverage for spark
> 2.0.0-SNAPSHOT to more in-house projects. today i ran into this issue...
>
> this runs fine:
> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
> df
> .map(row => row)(RowEncoder(df.schema))
> .select("a", "b")
> .show
>
> however this fails:
> val df = sc.parallelize(List(("1", "2"), ("3", "4"))).toDF("a", "b")
> df
> .map(row => row)(RowEncoder(df.schema))
> .select("b", "a")
> .show
>
> the error is:
> java.lang.Exception: failed to compile:
> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line
> 94, Column 57: Expression "mapelements_isNull" is not an rvalue
> /* 001 */ public Object generate(Object[] references) {
> /* 002 */ return new GeneratedIterator(references);
> /* 003 */ }
> /* 004 */
> /* 005 */ /** Codegened pipeline for:
> /* 006 */ * Project [b#11,a#10]
> /* 007 */ +- SerializeFromObject [if (input[0,
> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class org.ap...
> /* 008 */ */
> /* 009 */ final class GeneratedIterator extends
> org.apache.spark.sql.execution.BufferedRowIterator {
> /* 010 */ private Object[] references;
> /* 011 */ private scala.collection.Iterator inputadapter_input;
> /* 012 */ private UnsafeRow project_result;
> /* 013 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
> project_holder;
> /* 014 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
> project_rowWriter;
> /* 015 */ private Object[] deserializetoobject_values;
> /* 016 */ private org.apache.spark.sql.types.StructType
> deserializetoobject_schema;
> /* 017 */ private UnsafeRow deserializetoobject_result;
> /* 018 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
> deserializetoobject_holder;
> /* 019 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
> deserializetoobject_rowWriter;
> /* 020 */ private UnsafeRow mapelements_result;
> /* 021 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
> mapelements_holder;
> /* 022 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
> mapelements_rowWriter;
> /* 023 */ private UnsafeRow serializefromobject_result;
> /* 024 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
> serializefromobject_holder;
> /* 025 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
> serializefromobject_rowWriter;
> /* 026 */ private UnsafeRow project_result1;
> /* 027 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder
> project_holder1;
> /* 028 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
> project_rowWriter1;
> /* 029 */
> /* 030 */ public GeneratedIterator(Object[] references) {
> /* 031 */ this.references = references;
> /* 032 */ }
> /* 033 */
> /* 034 */ public void init(int index, scala.collection.Iterator
> inputs[]) {
> /* 035 */ partitionIndex = index;
> /* 036 */ inputadapter_input = inputs[0];
> /* 037 */ project_result = new UnsafeRow(2);
> /* 038 */ this.project_holder = new
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result,
> 64);
> /* 039 */ this.project_rowWriter = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder,
> 2);
> /* 040 */
> /* 041 */ this.deserializetoobject_schema =
> (org.apache.spark.sql.types.StructType) references[0];
> /* 042 */ deserializetoobject_result = new UnsafeRow(1);
> /* 043 */ this.deserializetoobject_holder = new
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result,
> 32);
> /* 044 */ this.deserializetoobject_rowWriter = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder,
> 1);
> /* 045 */ mapelements_result = new UnsafeRow(1);
> /* 046 */ this.mapelements_holder = new
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result,
> 32);
> /* 047 */ this.mapelements_rowWriter = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder,
> 1);
> /* 048 */ serializefromobject_result = new UnsafeRow(2);
> /* 049 */ this.serializefromobject_holder = new
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result,
> 64);
> /* 050 */ this.serializefromobject_rowWriter = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder,
> 2);
> /* 051 */ project_result1 = new UnsafeRow(2);
> /* 052 */ this.project_holder1 = new
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result1,
> 64);
> /* 053 */ this.project_rowWriter1 = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder1,
> 2);
> /* 054 */ }
> /* 055 */
> /* 056 */ protected void processNext() throws java.io.IOException {
> /* 057 */ /*** PRODUCE: Project [b#11,a#10] */
> /* 058 */
> /* 059 */ /*** PRODUCE: SerializeFromObject [if (input[0,
> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
> org.apache.spark.unsafe.types... */
> /* 060 */
> /* 061 */ /*** PRODUCE: MapElements <function1>, obj#9:
> org.apache.spark.sql.Row */
> /* 062 */
> /* 063 */ /*** PRODUCE: DeserializeToObject createexternalrow(if
> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
> b#6.toString, StructFi... */
> /* 064 */
> /* 065 */ /*** PRODUCE: Project [_1#2 AS a#5,_2#3 AS b#6] */
> /* 066 */
> /* 067 */ /*** PRODUCE: INPUT */
> /* 068 */
> /* 069 */ while (inputadapter_input.hasNext()) {
> /* 070 */ InternalRow inputadapter_row = (InternalRow)
> inputadapter_input.next();
> /* 071 */ /*** CONSUME: Project [_1#2 AS a#5,_2#3 AS b#6] */
> /* 072 */
> /* 073 */ /*** CONSUME: DeserializeToObject createexternalrow(if
> (isnull(a#5)) null else a#5.toString, if (isnull(b#6)) null else
> b#6.toString, StructFi... */
> /* 074 */ /* input[0, string] */
> /* 075 */ /* input[0, string] */
> /* 076 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
> /* 077 */ UTF8String inputadapter_value = inputadapter_isNull ? null
> : (inputadapter_row.getUTF8String(0));
> /* 078 */ /* input[1, string] */
> /* 079 */ /* input[1, string] */
> /* 080 */ boolean inputadapter_isNull1 =
> inputadapter_row.isNullAt(1);
> /* 081 */ UTF8String inputadapter_value1 = inputadapter_isNull1 ?
> null : (inputadapter_row.getUTF8String(1));
> /* 082 */
> /* 083 */ /*** CONSUME: MapElements <function1>, obj#9:
> org.apache.spark.sql.Row */
> /* 084 */
> /* 085 */ /*** CONSUME: SerializeFromObject [if (input[0,
> org.apache.spark.sql.Row].isNullAt) null else staticinvoke(class
> org.apache.spark.unsafe.types... */
> /* 086 */
> /* 087 */ /*** CONSUME: Project [b#11,a#10] */
> /* 088 */
> /* 089 */ /*** CONSUME: WholeStageCodegen */
> /* 090 */
> /* 091 */ /* input[1, string] */
> /* 092 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
> StringTy... */
> /* 093 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
> /* 094 */ boolean serializefromobject_isNull9 = mapelements_isNull
> || false;
> /* 095 */ final boolean serializefromobject_value9 =
> serializefromobject_isNull9 ? false : mapelements_value.isNullAt(1);
> /* 096 */ boolean serializefromobject_isNull8 = false;
> /* 097 */ UTF8String serializefromobject_value8 = null;
> /* 098 */ if (!serializefromobject_isNull9 &&
> serializefromobject_value9) {
> /* 099 */ /* null */
> /* 100 */ final UTF8String serializefromobject_value12 = null;
> /* 101 */ serializefromobject_isNull8 = true;
> /* 102 */ serializefromobject_value8 = serializefromobject_value12;
> /* 103 */ } else {
> /* 104 */ /* staticinvoke(class
> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
> getexternalrowfield(input[0, org.apache.spa... */
> /* 105 */ /* getexternalrowfield(input[0,
> org.apache.spark.sql.Row], 1, ObjectType(class java.lang.String)) */
> /* 106 */ if (mapelements_isNull) {
> /* 107 */ throw new RuntimeException("The input external row
> cannot be null.");
> /* 108 */ }
> /* 109 */
> /* 110 */ if (mapelements_value.isNullAt(1)) {
> /* 111 */ throw new RuntimeException("The 1th field of input row
> cannot be null.");
> /* 112 */ }
> /* 113 */
> /* 114 */ final java.lang.String serializefromobject_value14 =
> (java.lang.String) mapelements_value.get(1);
> /* 115 */ boolean serializefromobject_isNull13 = false;
> /* 116 */ final UTF8String serializefromobject_value13 =
> serializefromobject_isNull13 ? null :
> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value14);
> /* 117 */ serializefromobject_isNull13 =
> serializefromobject_value13 == null;
> /* 118 */ serializefromobject_isNull8 =
> serializefromobject_isNull13;
> /* 119 */ serializefromobject_value8 = serializefromobject_value13;
> /* 120 */ }
> /* 121 */ /* input[0, string] */
> /* 122 */ /* if (input[0, org.apache.spark.sql.Row].isNullAt) null
> else staticinvoke(class org.apache.spark.unsafe.types.UTF8String,
> StringTy... */
> /* 123 */ /* input[0, org.apache.spark.sql.Row].isNullAt */
> /* 124 */ /* input[0, org.apache.spark.sql.Row] */
> /* 125 */ /* <function1>.apply */
> /* 126 */ /* <function1> */
> /* 127 */ /* expression: <function1> */
> /* 128 */ Object mapelements_obj = ((Expression)
> references[1]).eval(null);
> /* 129 */ scala.Function1 mapelements_value1 = (scala.Function1)
> mapelements_obj;
> /* 130 */ /* input[0, org.apache.spark.sql.Row] */
> /* 131 */ /* createexternalrow(if (isnull(input[0, string])) null
> else input[0, string].toString, if (isnull(input[1, string])) null else
> inp... */
> /* 132 */ deserializetoobject_values = new Object[2];
> /* 133 */ /* if (isnull(input[0, string])) null else input[0,
> string].toString */
> /* 134 */ boolean deserializetoobject_isNull1 = false;
> /* 135 */ java.lang.String deserializetoobject_value1 = null;
> /* 136 */ if (!false && inputadapter_isNull) {
> /* 137 */ /* null */
> /* 138 */ final java.lang.String deserializetoobject_value4 = null;
> /* 139 */ deserializetoobject_isNull1 = true;
> /* 140 */ deserializetoobject_value1 = deserializetoobject_value4;
> /* 141 */ } else {
> /* 142 */ /* input[0, string].toString */
> /* 143 */ boolean deserializetoobject_isNull5 =
> inputadapter_isNull;
> /* 144 */ final java.lang.String deserializetoobject_value5 =
> deserializetoobject_isNull5 ? null : (java.lang.String)
> inputadapter_value.toString();
> /* 145 */ deserializetoobject_isNull5 = deserializetoobject_value5
> == null;
> /* 146 */ deserializetoobject_isNull1 =
> deserializetoobject_isNull5;
> /* 147 */ deserializetoobject_value1 = deserializetoobject_value5;
> /* 148 */ }
> /* 149 */ if (deserializetoobject_isNull1) {
> /* 150 */ deserializetoobject_values[0] = null;
> /* 151 */ } else {
> /* 152 */ deserializetoobject_values[0] =
> deserializetoobject_value1;
> /* 153 */ }
> /* 154 */ /* if (isnull(input[1, string])) null else input[1,
> string].toString */
> /* 155 */ boolean deserializetoobject_isNull7 = false;
> /* 156 */ java.lang.String deserializetoobject_value7 = null;
> /* 157 */ if (!false && inputadapter_isNull1) {
> /* 158 */ /* null */
> /* 159 */ final java.lang.String deserializetoobject_value10 =
> null;
> /* 160 */ deserializetoobject_isNull7 = true;
> /* 161 */ deserializetoobject_value7 = deserializetoobject_value10;
> /* 162 */ } else {
> /* 163 */ /* input[1, string].toString */
> /* 164 */ boolean deserializetoobject_isNull11 =
> inputadapter_isNull1;
> /* 165 */ final java.lang.String deserializetoobject_value11 =
> deserializetoobject_isNull11 ? null : (java.lang.String)
> inputadapter_value1.toString();
> /* 166 */ deserializetoobject_isNull11 =
> deserializetoobject_value11 == null;
> /* 167 */ deserializetoobject_isNull7 =
> deserializetoobject_isNull11;
> /* 168 */ deserializetoobject_value7 = deserializetoobject_value11;
> /* 169 */ }
> /* 170 */ if (deserializetoobject_isNull7) {
> /* 171 */ deserializetoobject_values[1] = null;
> /* 172 */ } else {
> /* 173 */ deserializetoobject_values[1] =
> deserializetoobject_value7;
> /* 174 */ }
> /* 175 */
> /* 176 */ final org.apache.spark.sql.Row deserializetoobject_value =
> new
> org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(deserializetoobject_values,
> this.deserializetoobject_schema);
> /* 177 */ boolean mapelements_isNull = false || false;
> /* 178 */ final org.apache.spark.sql.Row mapelements_value =
> mapelements_isNull ? null : (org.apache.spark.sql.Row)
> mapelements_value1.apply(deserializetoobject_value);
> /* 179 */ mapelements_isNull = mapelements_value == null;
> /* 180 */
> /* 181 */ boolean serializefromobject_isNull1 = mapelements_isNull
> || false;
> /* 182 */ final boolean serializefromobject_value1 =
> serializefromobject_isNull1 ? false : mapelements_value.isNullAt(0);
> /* 183 */ boolean serializefromobject_isNull = false;
> /* 184 */ UTF8String serializefromobject_value = null;
> /* 185 */ if (!serializefromobject_isNull1 &&
> serializefromobject_value1) {
> /* 186 */ /* null */
> /* 187 */ final UTF8String serializefromobject_value4 = null;
> /* 188 */ serializefromobject_isNull = true;
> /* 189 */ serializefromobject_value = serializefromobject_value4;
> /* 190 */ } else {
> /* 191 */ /* staticinvoke(class
> org.apache.spark.unsafe.types.UTF8String, StringType, fromString,
> getexternalrowfield(input[0, org.apache.spa... */
> /* 192 */ /* getexternalrowfield(input[0,
> org.apache.spark.sql.Row], 0, ObjectType(class java.lang.String)) */
> /* 193 */ if (mapelements_isNull) {
> /* 194 */ throw new RuntimeException("The input external row
> cannot be null.");
> /* 195 */ }
> /* 196 */
> /* 197 */ if (mapelements_value.isNullAt(0)) {
> /* 198 */ throw new RuntimeException("The 0th field of input row
> cannot be null.");
> /* 199 */ }
> /* 200 */
> /* 201 */ final java.lang.String serializefromobject_value6 =
> (java.lang.String) mapelements_value.get(0);
> /* 202 */ boolean serializefromobject_isNull5 = false;
> /* 203 */ final UTF8String serializefromobject_value5 =
> serializefromobject_isNull5 ? null :
> org.apache.spark.unsafe.types.UTF8String.fromString(serializefromobject_value6);
> /* 204 */ serializefromobject_isNull5 = serializefromobject_value5
> == null;
> /* 205 */ serializefromobject_isNull = serializefromobject_isNull5;
> /* 206 */ serializefromobject_value = serializefromobject_value5;
> /* 207 */ }
> /* 208 */ project_holder1.reset();
> /* 209 */
> /* 210 */ project_rowWriter1.zeroOutNullBytes();
> /* 211 */
> /* 212 */ if (serializefromobject_isNull8) {
> /* 213 */ project_rowWriter1.setNullAt(0);
> /* 214 */ } else {
> /* 215 */ project_rowWriter1.write(0, serializefromobject_value8);
> /* 216 */ }
> /* 217 */
> /* 218 */ if (serializefromobject_isNull) {
> /* 219 */ project_rowWriter1.setNullAt(1);
> /* 220 */ } else {
> /* 221 */ project_rowWriter1.write(1, serializefromobject_value);
> /* 222 */ }
> /* 223 */ project_result1.setTotalSize(project_holder1.totalSize());
> /* 224 */ append(project_result1);
> /* 225 */ if (shouldStop()) return;
> /* 226 */ }
> /* 227 */ }
> /* 228 */ }
>