Hi rakeshchalasani,

At the moment flink only splits methods by fields to avoid 64k problem, so
current implementation will reach the limits if a single field becomes too
large. Flink community has already planed to solve the problem, see [1]. As
a workaround, you can define you own udf to avoid the problem. The udf will
be serialized so it will not cause code bloat.

Best, Hequn

[1] https://issues.apache.org/jira/browse/FLINK-8920

On Tue, Jun 12, 2018 at 5:31 AM, rakeshchalasani <[email protected]>
wrote:

> Hi,
>
> We hit a situation where the code generation on Flink grows beyond 64KB and
> fails. Spark SQL has a similar issue and it automatically disables
> code-generation in such a case. Any way we can control that here? Following
> is the error stack:
>
> org.apache.flink.api.common.InvalidProgramException: Table program cannot
> be
> compiled. This is a bug. Please file an issue.
>         at org.apache.flink.table.codegen.Compiler$class.
> compile(Compiler.scala:36)
>         at
> org.apache.flink.table.runtime.CRowProcessRunner.
> compile(CRowProcessRunner.scala:35)
>         at
> org.apache.flink.table.runtime.CRowProcessRunner.
> open(CRowProcessRunner.scala:49)
>         at
> org.apache.flink.api.common.functions.util.FunctionUtils.
> openFunction(FunctionUtils.java:36)
>         at
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(
> AbstractUdfStreamOperator.java:111)
>         at
> org.apache.flink.streaming.api.operators.ProcessOperator.
> open(ProcessOperator.java:56)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.
> openAllOperators(StreamTask.java:376)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.
> invoke(StreamTask.java:253)
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.codehaus.janino.InternalCompilerException: Compiling
> "DataStreamCalcRule$20583": Code of method
> "processElement(Ljava/lang/Object;Lorg/apache/flink/
> streaming/api/functions/ProcessFunction$Context;Lorg/
> apache/flink/util/Collector;)V"
> of class "DataStreamCalcRule$20583" grows beyond 64 KB
>         at org.codehaus.janino.UnitCompiler.compileUnit(
> UnitCompiler.java:361)
>         at org.codehaus.janino.SimpleCompiler.cook(
> SimpleCompiler.java:234)
>         at
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(
> SimpleCompiler.java:446)
>         at org.codehaus.janino.SimpleCompiler.cook(
> SimpleCompiler.java:213)
>         at org.codehaus.janino.SimpleCompiler.cook(
> SimpleCompiler.java:204)
>         at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80)
>         at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75)
>         at org.apache.flink.table.codegen.Compiler$class.
> compile(Compiler.scala:33)
>         ... 9 common frames omitted
> Caused by: org.codehaus.janino.InternalCompilerException: Code of method
> "processElement(Ljava/lang/Object;Lorg/apache/flink/
> streaming/api/functions/ProcessFunction$Context;Lorg/
> apache/flink/util/Collector;)V"
> of class "DataStreamCalcRule$20583" grows beyond 64 KB
>         at org.codehaus.janino.CodeContext.makeSpace(CodeContext.java:990)
>         at org.codehaus.janino.CodeContext.write(CodeContext.java:867)
>         at org.codehaus.janino.UnitCompiler.writeOpcode(
> UnitCompiler.java:11901)
>         at org.codehaus.janino.UnitCompiler.store(UnitCompiler.java:11576)
>         at org.codehaus.janino.UnitCompiler.store(UnitCompiler.java:11560)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:2573)
>         at org.codehaus.janino.UnitCompiler.access$2700(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationS
> tatement(UnitCompiler.java:1482)
>         at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationS
> tatement(UnitCompiler.java:1466)
>         at
> org.codehaus.janino.Java$LocalVariableDeclarationStatem
> ent.accept(Java.java:3351)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:1532)
>         at org.codehaus.janino.UnitCompiler.access$1700(
> UnitCompiler.java:212)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1472)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.Java$Block.accept(Java.java:2756)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:2444)
>         at org.codehaus.janino.UnitCompiler.access$1900(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1474)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1466)
>         at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:1532)
>         at org.codehaus.janino.UnitCompiler.access$1700(
> UnitCompiler.java:212)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1472)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.Java$Block.accept(Java.java:2756)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:2444)
>         at org.codehaus.janino.UnitCompiler.access$1900(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1474)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1466)
>         at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:1532)
>         at org.codehaus.janino.UnitCompiler.access$1700(
> UnitCompiler.java:212)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1472)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.Java$Block.accept(Java.java:2756)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:2444)
>         at org.codehaus.janino.UnitCompiler.access$1900(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1474)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1466)
>         at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:1532)
>         at org.codehaus.janino.UnitCompiler.access$1700(
> UnitCompiler.java:212)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1472)
>         at org.codehaus.janino.UnitCompiler$6.visitBlock(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.Java$Block.accept(Java.java:2756)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:2444)
>         at org.codehaus.janino.UnitCompiler.access$1900(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1474)
>         at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.
> java:1466)
>         at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:1466)
>         at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
>         at org.codehaus.janino.UnitCompiler.compile(
> UnitCompiler.java:3075)
>         at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(
> UnitCompiler.java:1336)
>         at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(
> UnitCompiler.java:1309)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:799)
>         at org.codehaus.janino.UnitCompiler.compile2(
> UnitCompiler.java:411)
>         at org.codehaus.janino.UnitCompiler.access$400(
> UnitCompiler.java:212)
>         at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclara
> tion(UnitCompiler.java:390)
>         at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclara
> tion(UnitCompiler.java:385)
>         at
> org.codehaus.janino.Java$PackageMemberClassDeclaration.
> accept(Java.java:1405)
>         at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
>         at org.codehaus.janino.UnitCompiler.compileUnit(
> UnitCompiler.java:357)
>         ... 16 common frames omitted
>
>
>
>
>
> --
> Sent from: http://apache-flink-user-mailing-list-archive.2336050.
> n4.nabble.com/
>

Reply via email to