[jira] [Commented] (SPARK-19984) ERROR codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'

2017-10-05 Thread John Steidley (JIRA)

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

John Steidley commented on SPARK-19984:
---

[~kiszk] I noticed one other difference in our plans: yours has count(A#52) 
mine has count(1). Could that be significant?

> ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java'
> -
>
> Key: SPARK-19984
> URL: https://issues.apache.org/jira/browse/SPARK-19984
> Project: Spark
>  Issue Type: Bug
>  Components: Optimizer
>Affects Versions: 2.1.0
>Reporter: Andrey Yakovenko
> Attachments: after_adding_count.txt, before_adding_count.txt
>
>
> I had this error few time on my local hadoop 2.7.3+Spark2.1.0 environment. 
> This is not permanent error, next time i run it could disappear. 
> Unfortunately i don't know how to reproduce the issue.  As you can see from 
> the log my logic is pretty complicated.
> Here is a part of log i've got (container_1489514660953_0015_01_01)
> {code}
> 17/03/16 11:07:04 ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
> 151, Column 29: A method named "compare" is not declared in any enclosing 
> class nor any supertype, nor through a static import
> /* 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 boolean agg_initAgg;
> /* 009 */   private boolean agg_bufIsNull;
> /* 010 */   private long agg_bufValue;
> /* 011 */   private boolean agg_initAgg1;
> /* 012 */   private boolean agg_bufIsNull1;
> /* 013 */   private long agg_bufValue1;
> /* 014 */   private scala.collection.Iterator smj_leftInput;
> /* 015 */   private scala.collection.Iterator smj_rightInput;
> /* 016 */   private InternalRow smj_leftRow;
> /* 017 */   private InternalRow smj_rightRow;
> /* 018 */   private UTF8String smj_value2;
> /* 019 */   private java.util.ArrayList smj_matches;
> /* 020 */   private UTF8String smj_value3;
> /* 021 */   private UTF8String smj_value4;
> /* 022 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> smj_numOutputRows;
> /* 023 */   private UnsafeRow smj_result;
> /* 024 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder smj_holder;
> /* 025 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> smj_rowWriter;
> /* 026 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows;
> /* 027 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime;
> /* 028 */   private UnsafeRow agg_result;
> /* 029 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
> /* 030 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter;
> /* 031 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows1;
> /* 032 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime1;
> /* 033 */   private UnsafeRow agg_result1;
> /* 034 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
> /* 035 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter1;
> /* 036 */
> /* 037 */   public GeneratedIterator(Object[] references) {
> /* 038 */ this.references = references;
> /* 039 */   }
> /* 040 */
> /* 041 */   public void init(int index, scala.collection.Iterator[] inputs) {
> /* 042 */ partitionIndex = index;
> /* 043 */ this.inputs = inputs;
> /* 044 */ wholestagecodegen_init_0();
> /* 045 */ wholestagecodegen_init_1();
> /* 046 */
> /* 047 */   }
> /* 048 */
> /* 049 */   private void wholestagecodegen_init_0() {
> /* 050 */ agg_initAgg = false;
> /* 051 */
> /* 052 */ agg_initAgg1 = false;
> /* 053 */
> /* 054 */ smj_leftInput = inputs[0];
> /* 055 */ smj_rightInput = inputs[1];
> /* 056 */
> /* 057 */ smj_rightRow = null;
> /* 058 */
> /* 059 */ smj_matches = new java.util.ArrayList();
> /* 060 */
> /* 061 */ this.smj_numOutputRows = 
> (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
> /* 062 */ smj_result = new UnsafeRow(2);
> /* 063 */ this.smj_holder = new 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(smj_result, 
> 64);
> /* 064 */ this.smj_rowWriter = new 
> org.apache.spark.sql.catalyst.expression

[jira] [Commented] (SPARK-19984) ERROR codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'

2017-10-02 Thread John Steidley (JIRA)

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

John Steidley commented on SPARK-19984:
---

[~kiszk] Thank you for your quick response! I really appreciate your time 
helping me with this!

I noticed that this bug is more fragile than I reported yesterday. I can still 
reproduce it consistently, but I now know that it relies on two additional 
factors.

The first factor is applying a limit to the upstream dataframe (in my toy 
example, dfA). Without a limit the bug doesn't reproduce.

The second factor is printing the count of the final dataframe. If I don't ever 
make a call to .count() I also don't see the error.

Another thing to notice is that the error doesn't cause execution to stop. You 
can see in the "after_adding_count.txt" file there's a warning about not 
performing "Whole-stage codegen" but the execution does complete. As far as I 
know, the value is correct.

I think the most interesting lead to follow is the "Whole-stage" warning. That 
seems to include all the relevant things I've already mentioned: the limit, the 
nullability, and the count. Furthermore, when I attempt the reproduction in the 
console, the plan is different, for one thing it uses a different join, not a 
SortMergeJoin.

Does that help? Is there any other output or information I can get that would 
be useful?

> ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java'
> -
>
> Key: SPARK-19984
> URL: https://issues.apache.org/jira/browse/SPARK-19984
> Project: Spark
>  Issue Type: Bug
>  Components: Optimizer
>Affects Versions: 2.1.0
>Reporter: Andrey Yakovenko
> Attachments: after_adding_count.txt, before_adding_count.txt
>
>
> I had this error few time on my local hadoop 2.7.3+Spark2.1.0 environment. 
> This is not permanent error, next time i run it could disappear. 
> Unfortunately i don't know how to reproduce the issue.  As you can see from 
> the log my logic is pretty complicated.
> Here is a part of log i've got (container_1489514660953_0015_01_01)
> {code}
> 17/03/16 11:07:04 ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
> 151, Column 29: A method named "compare" is not declared in any enclosing 
> class nor any supertype, nor through a static import
> /* 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 boolean agg_initAgg;
> /* 009 */   private boolean agg_bufIsNull;
> /* 010 */   private long agg_bufValue;
> /* 011 */   private boolean agg_initAgg1;
> /* 012 */   private boolean agg_bufIsNull1;
> /* 013 */   private long agg_bufValue1;
> /* 014 */   private scala.collection.Iterator smj_leftInput;
> /* 015 */   private scala.collection.Iterator smj_rightInput;
> /* 016 */   private InternalRow smj_leftRow;
> /* 017 */   private InternalRow smj_rightRow;
> /* 018 */   private UTF8String smj_value2;
> /* 019 */   private java.util.ArrayList smj_matches;
> /* 020 */   private UTF8String smj_value3;
> /* 021 */   private UTF8String smj_value4;
> /* 022 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> smj_numOutputRows;
> /* 023 */   private UnsafeRow smj_result;
> /* 024 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder smj_holder;
> /* 025 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> smj_rowWriter;
> /* 026 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows;
> /* 027 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime;
> /* 028 */   private UnsafeRow agg_result;
> /* 029 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
> /* 030 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter;
> /* 031 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows1;
> /* 032 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime1;
> /* 033 */   private UnsafeRow agg_result1;
> /* 034 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
> /* 035 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter1;
> /* 036 */
> /* 037 */   public GeneratedIterator(Object[] references) 

[jira] [Updated] (SPARK-19984) ERROR codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'

2017-10-02 Thread John Steidley (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-19984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

John Steidley updated SPARK-19984:
--
Attachment: after_adding_count.txt
before_adding_count.txt

> ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java'
> -
>
> Key: SPARK-19984
> URL: https://issues.apache.org/jira/browse/SPARK-19984
> Project: Spark
>  Issue Type: Bug
>  Components: Optimizer
>Affects Versions: 2.1.0
>Reporter: Andrey Yakovenko
> Attachments: after_adding_count.txt, before_adding_count.txt
>
>
> I had this error few time on my local hadoop 2.7.3+Spark2.1.0 environment. 
> This is not permanent error, next time i run it could disappear. 
> Unfortunately i don't know how to reproduce the issue.  As you can see from 
> the log my logic is pretty complicated.
> Here is a part of log i've got (container_1489514660953_0015_01_01)
> {code}
> 17/03/16 11:07:04 ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
> 151, Column 29: A method named "compare" is not declared in any enclosing 
> class nor any supertype, nor through a static import
> /* 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 boolean agg_initAgg;
> /* 009 */   private boolean agg_bufIsNull;
> /* 010 */   private long agg_bufValue;
> /* 011 */   private boolean agg_initAgg1;
> /* 012 */   private boolean agg_bufIsNull1;
> /* 013 */   private long agg_bufValue1;
> /* 014 */   private scala.collection.Iterator smj_leftInput;
> /* 015 */   private scala.collection.Iterator smj_rightInput;
> /* 016 */   private InternalRow smj_leftRow;
> /* 017 */   private InternalRow smj_rightRow;
> /* 018 */   private UTF8String smj_value2;
> /* 019 */   private java.util.ArrayList smj_matches;
> /* 020 */   private UTF8String smj_value3;
> /* 021 */   private UTF8String smj_value4;
> /* 022 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> smj_numOutputRows;
> /* 023 */   private UnsafeRow smj_result;
> /* 024 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder smj_holder;
> /* 025 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> smj_rowWriter;
> /* 026 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows;
> /* 027 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime;
> /* 028 */   private UnsafeRow agg_result;
> /* 029 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
> /* 030 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter;
> /* 031 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows1;
> /* 032 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime1;
> /* 033 */   private UnsafeRow agg_result1;
> /* 034 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
> /* 035 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter1;
> /* 036 */
> /* 037 */   public GeneratedIterator(Object[] references) {
> /* 038 */ this.references = references;
> /* 039 */   }
> /* 040 */
> /* 041 */   public void init(int index, scala.collection.Iterator[] inputs) {
> /* 042 */ partitionIndex = index;
> /* 043 */ this.inputs = inputs;
> /* 044 */ wholestagecodegen_init_0();
> /* 045 */ wholestagecodegen_init_1();
> /* 046 */
> /* 047 */   }
> /* 048 */
> /* 049 */   private void wholestagecodegen_init_0() {
> /* 050 */ agg_initAgg = false;
> /* 051 */
> /* 052 */ agg_initAgg1 = false;
> /* 053 */
> /* 054 */ smj_leftInput = inputs[0];
> /* 055 */ smj_rightInput = inputs[1];
> /* 056 */
> /* 057 */ smj_rightRow = null;
> /* 058 */
> /* 059 */ smj_matches = new java.util.ArrayList();
> /* 060 */
> /* 061 */ this.smj_numOutputRows = 
> (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
> /* 062 */ smj_result = new UnsafeRow(2);
> /* 063 */ this.smj_holder = new 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(smj_result, 
> 64);
> /* 064 */ this.smj_rowWriter = new 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(smj_holder, 
> 2);
> /* 065 */ this.agg_numOutputRows = 
> (org.apache.s

[jira] [Commented] (SPARK-19984) ERROR codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'

2017-10-01 Thread John Steidley (JIRA)

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

John Steidley commented on SPARK-19984:
---

[~kiszk] I just hit this issue. I am able to reproduce it consistently on Spark 
2.1.1 from inside my codebase, but not in the spark shell.

Here's the a rough equivalent of my code (again, this does not reproduce the 
bug in the spark console)
```
val dfA = sc.parallelize(Seq("John", "Kazuaki")).toDF("id")

val dfB = dfA.select(col("id").alias("A"), col("id").alias("B"))

val dfC = sc.parallelize(Seq("John", "Kazuaki")).toDF("A")
dfC.join(dfB, "A")
```

I have though about nullability, making dfA's "id" column non-nullable makes 
the issue go away. (following the directions here 
https://stackoverflow.com/questions/33193958/change-nullable-property-of-column-in-spark-dataframe)

In my real code dfA and dfC are related (both are derived from the same 
dataframe), could that matter here?

What could I be missing? Is there any more information you would like from me? 
I can try harder to track down the issue, but I can't share the whole codebase.

> ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java'
> -
>
> Key: SPARK-19984
> URL: https://issues.apache.org/jira/browse/SPARK-19984
> Project: Spark
>  Issue Type: Bug
>  Components: Optimizer
>Affects Versions: 2.1.0
>Reporter: Andrey Yakovenko
>
> I had this error few time on my local hadoop 2.7.3+Spark2.1.0 environment. 
> This is not permanent error, next time i run it could disappear. 
> Unfortunately i don't know how to reproduce the issue.  As you can see from 
> the log my logic is pretty complicated.
> Here is a part of log i've got (container_1489514660953_0015_01_01)
> {code}
> 17/03/16 11:07:04 ERROR codegen.CodeGenerator: failed to compile: 
> org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
> 151, Column 29: A method named "compare" is not declared in any enclosing 
> class nor any supertype, nor through a static import
> /* 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 boolean agg_initAgg;
> /* 009 */   private boolean agg_bufIsNull;
> /* 010 */   private long agg_bufValue;
> /* 011 */   private boolean agg_initAgg1;
> /* 012 */   private boolean agg_bufIsNull1;
> /* 013 */   private long agg_bufValue1;
> /* 014 */   private scala.collection.Iterator smj_leftInput;
> /* 015 */   private scala.collection.Iterator smj_rightInput;
> /* 016 */   private InternalRow smj_leftRow;
> /* 017 */   private InternalRow smj_rightRow;
> /* 018 */   private UTF8String smj_value2;
> /* 019 */   private java.util.ArrayList smj_matches;
> /* 020 */   private UTF8String smj_value3;
> /* 021 */   private UTF8String smj_value4;
> /* 022 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> smj_numOutputRows;
> /* 023 */   private UnsafeRow smj_result;
> /* 024 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder smj_holder;
> /* 025 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> smj_rowWriter;
> /* 026 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows;
> /* 027 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime;
> /* 028 */   private UnsafeRow agg_result;
> /* 029 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
> /* 030 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter;
> /* 031 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_numOutputRows1;
> /* 032 */   private org.apache.spark.sql.execution.metric.SQLMetric 
> agg_aggTime1;
> /* 033 */   private UnsafeRow agg_result1;
> /* 034 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
> /* 035 */   private 
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter 
> agg_rowWriter1;
> /* 036 */
> /* 037 */   public GeneratedIterator(Object[] references) {
> /* 038 */ this.references = references;
> /* 039 */   }
> /* 040 */
> /* 041 */   public void init(int index, scala.collection.Iterator[] inputs) {
> /* 042 */ partitionIndex = index;
> /* 043 */ this.inputs = inputs;
> /* 044 */ wholestagecodegen_init_0();
> /* 045 */ wholestagecodegen_init_1();
> /* 046 */
>