[jira] [Comment Edited] (SPARK-36862) ERROR CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'
[ https://issues.apache.org/jira/browse/SPARK-36862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17599013#comment-17599013 ] Lukas Waldmann edited comment on SPARK-36862 at 9/1/22 2:55 PM: I managed to reproduce the issue in my environment. Problem is on line 192 - variable name in function header having array index Here is the generated code {code:java} /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage636(references); /* 003 */ } /* 004 */ /* 005 */ // codegenStageId=636 /* 006 */ final class GeneratedIteratorForCodegenStage636 extends org.apache.spark.sql.execution.BufferedRowIterator { /* 007 */ private Object[] references; /* 008 */ private scala.collection.Iterator[] inputs; /* 009 */ private scala.collection.Iterator smj_leftInput_0; /* 010 */ private scala.collection.Iterator smj_rightInput_0; /* 011 */ private InternalRow smj_leftRow_0; /* 012 */ private InternalRow smj_rightRow_0; /* 013 */ private boolean smj_globalIsNull_0; /* 014 */ private boolean smj_globalIsNull_1; /* 015 */ private double smj_value_27; /* 016 */ private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0; /* 017 */ private double smj_value_28; /* 018 */ private boolean smj_isNull_25; /* 019 */ private boolean smj_isNull_26; /* 020 */ private boolean smj_isNull_27; /* 021 */ private boolean smj_isNull_28; /* 022 */ private boolean smj_isNull_29; /* 023 */ private boolean smj_isNull_30; /* 024 */ private boolean project_subExprIsNull_0; /* 025 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_2 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2]; /* 026 */ private java.util.regex.Pattern[] project_mutableStateArray_0 = new java.util.regex.Pattern[1]; /* 027 */ private Decimal[] smj_mutableStateArray_1 = new Decimal[1]; /* 028 */ private String[] project_mutableStateArray_1 = new String[1]; /* 029 */ private UTF8String[] smj_mutableStateArray_0 = new UTF8String[7]; /* 030 */ /* 031 */ public GeneratedIteratorForCodegenStage636(Object[] references) { /* 032 */ this.references = references; /* 033 */ } /* 034 */ /* 035 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 036 */ partitionIndex = index; /* 037 */ this.inputs = inputs; /* 038 */ smj_leftInput_0 = inputs[0]; /* 039 */ smj_rightInput_0 = inputs[1]; /* 040 */ /* 041 */ smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(2147483632, 2147483647); /* 042 */ smj_mutableStateArray_2[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(6, 192); /* 043 */ smj_mutableStateArray_2[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(6, 192); /* 044 */ /* 045 */ } /* 046 */ /* 047 */ private boolean smj_findNextOuterJoinRows_0( /* 048 */ scala.collection.Iterator leftIter, /* 049 */ scala.collection.Iterator rightIter) { /* 050 */ smj_leftRow_0 = null; /* 051 */ int comp = 0; /* 052 */ while (smj_leftRow_0 == null) { /* 053 */ if (!leftIter.hasNext()) return false; /* 054 */ smj_leftRow_0 = (InternalRow) leftIter.next(); /* 055 */ UTF8String smj_value_22 = smj_If_0(smj_leftRow_0); /* 056 */ boolean smj_isNull_2 = smj_globalIsNull_1; /* 057 */ double smj_value_2 = -1.0; /* 058 */ if (!smj_globalIsNull_1) { /* 059 */ final String smj_doubleStr_0 = smj_value_22.toString(); /* 060 */ try { /* 061 */ smj_value_2 = Double.valueOf(smj_doubleStr_0); /* 062 */ } catch (java.lang.NumberFormatException e) { /* 063 */ final Double d = (Double) Cast.processFloatingPointSpecialLiterals(smj_doubleStr_0, false); /* 064 */ if (d == null) { /* 065 */ smj_isNull_2 = true; /* 066 */ } else { /* 067 */ smj_value_2 = d.doubleValue(); /* 068 */ } /* 069 */ } /* 070 */ } /* 071 */ boolean smj_isNull_1 = smj_isNull_2; /* 072 */ double smj_value_1 = -1.0; /* 073 */ /* 074 */ if (!smj_isNull_2) { /* 075 */ if (Double.isNaN(smj_value_2)) { /* 076 */ smj_value_1 = Double.NaN; /* 077 */ } else if (smj_value_2 == -0.0d) { /* 078 */ smj_value_1 = 0.0d; /* 079 */ } else { /* 080 */ smj_value_1 = smj_value_2; /* 081 */ } /* 082 */ /* 083 */ } /* 084 */ if (smj_isNull_1) { /* 085 */ if (!smj_matches_0.isEmpty()) { /* 086 */ smj_matches_0.clear(); /* 087 */ } /* 088 */ return true; /* 089 */ } /* 090 */ if (!smj_matches_0.isEmpty()) { /* 091 */ comp = 0; /* 092 */ if (comp == 0) { /* 093 */ comp = org.apache.spark.sql.catalyst.util.SQLOrderingUtil.compareDoubles(smj_value_1, smj_value_28); /* 094 */ } /* 095 */ /* 096 */ if (comp == 0) { /* 097 */ return true; /* 098 */ } /* 099 */ smj_matches_0.clear(); /* 100 */ } /* 101 */ /* 102 */ do { /* 103 */ if (smj_rightRow_0 == null) { /* 104 */ if (!rightIter.hasNext()) { /* 105 */ if (!smj_matches_0.isEmpty()) { /* 106 */ smj_value_28 = smj_value_1; /* 107 */ } /* 108 */ return true; /*
[jira] [Comment Edited] (SPARK-36862) ERROR CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java'
[ https://issues.apache.org/jira/browse/SPARK-36862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17421374#comment-17421374 ] Magdalena Pilawska edited comment on SPARK-36862 at 9/28/21, 1:11 PM: -- I get the physical execution plan as a part of output log but I cannot share that in public if you mean so. Any thoughts why the same works on spark 3.0.0? was (Author: mpilaw): I get the physical execution plan as a part of output log but I cannot share that in public if you mean so. Any thoughts why the same works on 3.0.0? > ERROR CodeGenerator: failed to compile: > org.codehaus.commons.compiler.CompileException: File 'generated.java' > - > > Key: SPARK-36862 > URL: https://issues.apache.org/jira/browse/SPARK-36862 > Project: Spark > Issue Type: Bug > Components: Spark Submit, SQL >Affects Versions: 3.1.1, 3.1.2 > Environment: Spark 3.1.1 and Spark 3.1.2 > hadoop 3.2.1 >Reporter: Magdalena Pilawska >Priority: Major > > Hi, > I am getting the following error running spark-submit command: > ERROR CodeGenerator: failed to compile: > org.codehaus.commons.compiler.CompileException: File 'generated.java', Line > 321, Column 103: ')' expected instead of '[' > > It fails running the spark sql command on delta lake: > spark.sql(sqlTransformation) > The template of sqlTransformation is as follows: > MERGE INTO target_table AS d > USING source_table AS s > on s.id = d.id > WHEN MATCHED AND d.hash_value <> s.hash_value > THEN UPDATE SET d.name =s.name, d.address = s.address > > It is permanent error both for *spark 3.1.1* and *3.1.2* versions. > > The same works fine with spark 3.0.0. > > Here is the full log: > 2021-09-22 16:43:22,110 ERROR CodeGenerator: failed to compile: > org.codehaus.commons.compiler.CompileException: File 'generated.java', Line > 55, Column 103: ')' expected instead of '['2021-09-22 16:43:22,110 ERROR > CodeGenerator: failed to compile: > org.codehaus.commons.compiler.CompileException: File 'generated.java', Line > 55, Column 103: ')' expected instead of > '['org.codehaus.commons.compiler.CompileException: File 'generated.java', > Line 55, Column 103: ')' expected instead of '[' at > org.codehaus.janino.TokenStreamImpl.compileException(TokenStreamImpl.java:362) > at org.codehaus.janino.TokenStreamImpl.read(TokenStreamImpl.java:150) at > org.codehaus.janino.Parser.read(Parser.java:3703) at > org.codehaus.janino.Parser.parseFormalParameters(Parser.java:1622) at > org.codehaus.janino.Parser.parseMethodDeclarationRest(Parser.java:1518) at > org.codehaus.janino.Parser.parseClassBodyDeclaration(Parser.java:1028) at > org.codehaus.janino.Parser.parseClassBody(Parser.java:841) at > org.codehaus.janino.Parser.parseClassDeclarationRest(Parser.java:736) at > org.codehaus.janino.Parser.parseClassBodyDeclaration(Parser.java:941) at > org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:234) at > org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:205) at > org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80) at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1427) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1524) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1521) > at > org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599) > at > org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379) > at > org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342) > at org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257) > at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000) at > org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004) at > org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1375) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.liftedTree1$1(WholeStageCodegenExec.scala:721) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:720) > at > org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:185) > at > org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:223) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution