[jira] [Comment Edited] (SPARK-18492) GeneratedIterator grows beyond 64 KB

2018-03-02 Thread Kazuaki Ishizaki (JIRA)

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

Kazuaki Ishizaki edited comment on SPARK-18492 at 3/3/18 2:08 AM:
--

Thanks, what is the definition and sample input of {{truetradescast}}?


was (Author: kiszk):
Thanks, what is the definition of {{truetradescast}}?

> GeneratedIterator grows beyond 64 KB
> 
>
> Key: SPARK-18492
> URL: https://issues.apache.org/jira/browse/SPARK-18492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.1
> Environment: CentOS release 6.7 (Final)
>Reporter: Norris Merritt
>Priority: Major
> Attachments: Screenshot from 2018-03-02 12-57-51.png
>
>
> spark-submit fails with ERROR CodeGenerator: failed to compile: 
> org.codehaus.janino.JaninoRuntimeException: Code of method 
> "(I[Lscala/collection/Iterator;)V" of class 
> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
> grows beyond 64 KB
> Error message is followed by a huge dump of generated source code.
> The generated code declares 1,454 field sequences like the following:
> /* 036 */   private org.apache.spark.sql.catalyst.expressions.ScalaUDF 
> project_scalaUDF1;
> /* 037 */   private scala.Function1 project_catalystConverter1;
> /* 038 */   private scala.Function1 project_converter1;
> /* 039 */   private scala.Function1 project_converter2;
> /* 040 */   private scala.Function2 project_udf1;
>   (many omitted lines) ...
> /* 6089 */   private org.apache.spark.sql.catalyst.expressions.ScalaUDF 
> project_scalaUDF1454;
> /* 6090 */   private scala.Function1 project_catalystConverter1454;
> /* 6091 */   private scala.Function1 project_converter1695;
> /* 6092 */   private scala.Function1 project_udf1454;
> It then proceeds to emit code for several methods (init, processNext) each of 
> which has totally repetitive sequences of statements pertaining to each of 
> the sequences of variables declared in the class.  For example:
> /* 6101 */   public void init(int index, scala.collection.Iterator inputs[]) {
> The reason that the 64KB JVM limit for code for a method is exceeded is 
> because the code generator is using an incredibly naive strategy.  It emits a 
> sequence like the one shown below for each of the 1,454 groups of variables 
> shown above, in 
> /* 6132 */ this.project_udf = 
> (scala.Function1)project_scalaUDF.userDefinedFunc();
> /* 6133 */ this.project_scalaUDF1 = 
> (org.apache.spark.sql.catalyst.expressions.ScalaUDF) references[10];
> /* 6134 */ this.project_catalystConverter1 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToCatalystConverter(project_scalaUDF1.dataType());
> /* 6135 */ this.project_converter1 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToScalaConverter(((org.apache.spark.sql.catalyst.expressions.Expression)(((org.apache.spark.sql.catalyst.expressions.ScalaUDF)references[10]).getChildren().apply(0))).dataType());
> /* 6136 */ this.project_converter2 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToScalaConverter(((org.apache.spark.sql.catalyst.expressions.Expression)(((org.apache.spark.sql.catalyst.expressions.ScalaUDF)references[10]).getChildren().apply(1))).dataType());
> It blows up after emitting 230 such sequences, while trying to emit the 231st:
> /* 7282 */ this.project_udf230 = 
> (scala.Function2)project_scalaUDF230.userDefinedFunc();
> /* 7283 */ this.project_scalaUDF231 = 
> (org.apache.spark.sql.catalyst.expressions.ScalaUDF) references[240];
> /* 7284 */ this.project_catalystConverter231 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToCatalystConverter(project_scalaUDF231.dataType());
>   many omitted lines ...
>  Example of repetitive code sequences emitted for processNext method:
> /* 12253 */   boolean project_isNull247 = project_result244 == null;
> /* 12254 */   MapData project_value247 = null;
> /* 12255 */   if (!project_isNull247) {
> /* 12256 */ project_value247 = project_result244;
> /* 12257 */   }
> /* 12258 */   Object project_arg = sort_isNull5 ? null : 
> project_converter489.apply(sort_value5);
> /* 12259 */
> /* 12260 */   ArrayData project_result249 = null;
> /* 12261 */   try {
> /* 12262 */ project_result249 = 
> (ArrayData)project_catalystConverter248.apply(project_udf248.apply(project_arg));
> /* 12263 */   } catch (Exception e) {
> /* 12264 */ throw new 
> org.apache.spark.SparkException(project_scalaUDF248.udfErrorMessage(), e);
> /* 12265 */   }
> /* 12266 */
> /* 12267 */   boolean project_isNull252 = 

[jira] [Comment Edited] (SPARK-18492) GeneratedIterator grows beyond 64 KB

2016-12-19 Thread Roberto Mirizzi (JIRA)

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

Roberto Mirizzi edited comment on SPARK-18492 at 12/19/16 11:32 PM:


I'm having exactly the same issue on Spark 2.0.2. I had it also on Spark 2.0.0 
and Spark 2.0.1.
My exception is:

{{JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB}}

It happens when I try to do many transformations on a given dataset, like 
multiple {{.select(...).select(...)}} with multiple operations inside the 
select (like {{when(...).otherwise(...)}}, or {{regexp_extract}}).

The exception outputs about 15k lines of Java code, like:

{code:java}
16/12/19 07:16:43 ERROR CodeGenerator: failed to compile: 
org.codehaus.janino.JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB
/* 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 boolean agg_initAgg;
/* 008 */   private org.apache.spark.sql.execution.aggregate.HashAggregateExec 
agg_plan;
/* 009 */   private 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 010 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter 
agg_sorter;
/* 011 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_peakMemory;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_spillSize;
/* 014 */   private scala.collection.Iterator inputadapter_input;
/* 015 */   private UnsafeRow agg_result;
/* 016 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 017 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 018 */   private UTF8String agg_lastRegex;
{code}

However, it looks like the execution continues successfully.

This is part of the stack trace:
{code:java}
org.codehaus.janino.JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB
at org.codehaus.janino.CodeContext.makeSpace(CodeContext.java:941)
at org.codehaus.janino.CodeContext.write(CodeContext.java:854)
at org.codehaus.janino.CodeContext.writeShort(CodeContext.java:959)
at 
org.codehaus.janino.UnitCompiler.writeConstantClassInfo(UnitCompiler.java:10274)
at 
org.codehaus.janino.UnitCompiler.tryNarrowingReferenceConversion(UnitCompiler.java:9725)
at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3833)
at org.codehaus.janino.UnitCompiler.access$6400(UnitCompiler.java:185)
at org.codehaus.janino.UnitCompiler$10.visitCast(UnitCompiler.java:3258)
at org.codehaus.janino.Java$Cast.accept(Java.java:3802)
at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:3290)
at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3868)
at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:185)
at 
org.codehaus.janino.UnitCompiler$10.visitParenthesizedExpression(UnitCompiler.java:3286)
{code}

And after that I get a warning:
{code:java}
16/12/19 07:16:45 WARN WholeStageCodegenExec: Whole-stage codegen disabled for 
this plan:
 *HashAggregate(keys=...
{code}


was (Author: roberto.mirizzi):
I'm having exactly the same issue on Spark 2.0.2. I had it also on Spark 2.0.0 
and Spark 2.0.1.
My exception is:

{{JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB}}

It happens when I try to do many transformations on a given dataset, like 
multiple {{.select(...).select(...)}} with multiple operations inside the 
select (like {{when(...).otherwise(...)}}, or {{regexp_extract}}).

The exception outputs about 15k lines of Java code, like:

{code:java}
16/12/19 07:16:43 ERROR CodeGenerator: failed to compile: 
org.codehaus.janino.JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB
/* 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 boolean agg_initAgg;
/* 008 */   private 

[jira] [Comment Edited] (SPARK-18492) GeneratedIterator grows beyond 64 KB

2016-12-19 Thread Roberto Mirizzi (JIRA)

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

Roberto Mirizzi edited comment on SPARK-18492 at 12/19/16 11:29 PM:


I'm having exactly the same issue on Spark 2.0.2. I had it also on Spark 2.0.0 
and Spark 2.0.1.
My exception is:

{{JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB}}

It happens when I try to do many transformations on a given dataset, like 
multiple {{.select(...).select(...)}} with multiple operations inside the 
select (like {{when(...).otherwise(...)}}, or {{regexp_extract}}).

The exception outputs about 15k lines of Java code, like:

{code:java}
16/12/19 07:16:43 ERROR CodeGenerator: failed to compile: 
org.codehaus.janino.JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB
/* 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 boolean agg_initAgg;
/* 008 */   private org.apache.spark.sql.execution.aggregate.HashAggregateExec 
agg_plan;
/* 009 */   private 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 010 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter 
agg_sorter;
/* 011 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_peakMemory;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_spillSize;
/* 014 */   private scala.collection.Iterator inputadapter_input;
/* 015 */   private UnsafeRow agg_result;
/* 016 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 017 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 018 */   private UTF8String agg_lastRegex;
{code}

However, it looks like the execution continues successfully.



was (Author: roberto.mirizzi):
I'm having exactly the same issue on Spark 2.0.2. I had it also on Spark 2.0.0 
and Spark 2.0.1.
My exception is:

bq. JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB

It happens when I try to do many transformations on a given dataset, like 
multiple {{.select(...).select(...)}} with multiple operations inside the 
select (like {{when(...).otherwise(...)}}, or {{regexp_extract}}).

The exception outputs about 15k lines of Java code, like:

{code:java}
16/12/19 07:16:43 ERROR CodeGenerator: failed to compile: 
org.codehaus.janino.JaninoRuntimeException: Code of method "()V" of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
grows beyond 64 KB
/* 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 boolean agg_initAgg;
/* 008 */   private org.apache.spark.sql.execution.aggregate.HashAggregateExec 
agg_plan;
/* 009 */   private 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 010 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter 
agg_sorter;
/* 011 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_peakMemory;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric 
agg_spillSize;
/* 014 */   private scala.collection.Iterator inputadapter_input;
/* 015 */   private UnsafeRow agg_result;
/* 016 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 017 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 018 */   private UTF8String agg_lastRegex;
{code}

However, it looks like the execution continues successfully.


> GeneratedIterator grows beyond 64 KB
> 
>
> Key: SPARK-18492
> URL: https://issues.apache.org/jira/browse/SPARK-18492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.1
> Environment: CentOS release 6.7 (Final)
>Reporter: Norris Merritt
>
> spark-submit fails with ERROR CodeGenerator: failed to compile: 
> org.codehaus.janino.JaninoRuntimeException: Code of method 
> "(I[Lscala/collection/Iterator;)V" of class 
> 

[jira] [Comment Edited] (SPARK-18492) GeneratedIterator grows beyond 64 KB

2016-11-17 Thread Kazuaki Ishizaki (JIRA)

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

Kazuaki Ishizaki edited comment on SPARK-18492 at 11/17/16 7:31 PM:


I agree with your point. Can you post a small program that can reproduce this 
issue?


was (Author: kiszk):
Can you post a small program that can reproduce this issue?

> GeneratedIterator grows beyond 64 KB
> 
>
> Key: SPARK-18492
> URL: https://issues.apache.org/jira/browse/SPARK-18492
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.1
> Environment: CentOS release 6.7 (Final)
>Reporter: Norris Merritt
>
> spark-submit fails with ERROR CodeGenerator: failed to compile: 
> org.codehaus.janino.JaninoRuntimeException: Code of method 
> "(I[Lscala/collection/Iterator;)V" of class 
> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator" 
> grows beyond 64 KB
> Error message is followed by a huge dump of generated source code.
> The generated code declares 1,454 field sequences like the following:
> /* 036 */   private org.apache.spark.sql.catalyst.expressions.ScalaUDF 
> project_scalaUDF1;
> /* 037 */   private scala.Function1 project_catalystConverter1;
> /* 038 */   private scala.Function1 project_converter1;
> /* 039 */   private scala.Function1 project_converter2;
> /* 040 */   private scala.Function2 project_udf1;
>   (many omitted lines) ...
> /* 6089 */   private org.apache.spark.sql.catalyst.expressions.ScalaUDF 
> project_scalaUDF1454;
> /* 6090 */   private scala.Function1 project_catalystConverter1454;
> /* 6091 */   private scala.Function1 project_converter1695;
> /* 6092 */   private scala.Function1 project_udf1454;
> It then proceeds to emit code for several methods (init, processNext) each of 
> which has totally repetitive sequences of statements pertaining to each of 
> the sequences of variables declared in the class.  For example:
> /* 6101 */   public void init(int index, scala.collection.Iterator inputs[]) {
> The reason that the 64KB JVM limit for code for a method is exceeded is 
> because the code generator is using an incredibly naive strategy.  It emits a 
> sequence like the one shown below for each of the 1,454 groups of variables 
> shown above, in 
> /* 6132 */ this.project_udf = 
> (scala.Function1)project_scalaUDF.userDefinedFunc();
> /* 6133 */ this.project_scalaUDF1 = 
> (org.apache.spark.sql.catalyst.expressions.ScalaUDF) references[10];
> /* 6134 */ this.project_catalystConverter1 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToCatalystConverter(project_scalaUDF1.dataType());
> /* 6135 */ this.project_converter1 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToScalaConverter(((org.apache.spark.sql.catalyst.expressions.Expression)(((org.apache.spark.sql.catalyst.expressions.ScalaUDF)references[10]).getChildren().apply(0))).dataType());
> /* 6136 */ this.project_converter2 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToScalaConverter(((org.apache.spark.sql.catalyst.expressions.Expression)(((org.apache.spark.sql.catalyst.expressions.ScalaUDF)references[10]).getChildren().apply(1))).dataType());
> It blows up after emitting 230 such sequences, while trying to emit the 231st:
> /* 7282 */ this.project_udf230 = 
> (scala.Function2)project_scalaUDF230.userDefinedFunc();
> /* 7283 */ this.project_scalaUDF231 = 
> (org.apache.spark.sql.catalyst.expressions.ScalaUDF) references[240];
> /* 7284 */ this.project_catalystConverter231 = 
> (scala.Function1)org.apache.spark.sql.catalyst.CatalystTypeConverters$.MODULE$.createToCatalystConverter(project_scalaUDF231.dataType());
>   many omitted lines ...
>  Example of repetitive code sequences emitted for processNext method:
> /* 12253 */   boolean project_isNull247 = project_result244 == null;
> /* 12254 */   MapData project_value247 = null;
> /* 12255 */   if (!project_isNull247) {
> /* 12256 */ project_value247 = project_result244;
> /* 12257 */   }
> /* 12258 */   Object project_arg = sort_isNull5 ? null : 
> project_converter489.apply(sort_value5);
> /* 12259 */
> /* 12260 */   ArrayData project_result249 = null;
> /* 12261 */   try {
> /* 12262 */ project_result249 = 
> (ArrayData)project_catalystConverter248.apply(project_udf248.apply(project_arg));
> /* 12263 */   } catch (Exception e) {
> /* 12264 */ throw new 
> org.apache.spark.SparkException(project_scalaUDF248.udfErrorMessage(), e);
> /* 12265 */   }
> /* 12266 */
> /* 12267 */   boolean project_isNull252 = project_result249 == null;
> /* 12268 */   ArrayData project_value252 =