You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "jack sun (Jira)" <ji...@apache.org> on 2020/08/13 12:12:00 UTC

[jira] [Created] (FLINK-18940) Cannot cast "org.apache.flink.table.data.RowData" to "org.apache.flink.table.data.binary.BinaryStringData"

jack sun created FLINK-18940:
--------------------------------

             Summary: Cannot cast "org.apache.flink.table.data.RowData" to "org.apache.flink.table.data.binary.BinaryStringData"
                 Key: FLINK-18940
                 URL: https://issues.apache.org/jira/browse/FLINK-18940
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Runtime
    Affects Versions: 1.11.1
         Environment: macOS Catalina
java8
scala 2.11
            Reporter: jack sun


/* 1 */
/* 2 */      public class SinkConversion$4 extends org.apache.flink.table.runtime.operators.TableStreamOperator
/* 3 */          implements org.apache.flink.streaming.api.operators.OneInputStreamOperator {
/* 4 */
/* 5 */        private final Object[] references;
/* 6 */        private transient org.apache.flink.table.data.util.DataFormatConverters.StringConverter converter$3;
/* 7 */        private final org.apache.flink.streaming.runtime.streamrecord.StreamRecord outElement = new org.apache.flink.streaming.runtime.streamrecord.StreamRecord(null);
/* 8 */
/* 9 */        public SinkConversion$4(
/* 10 */            Object[] references,
/* 11 */            org.apache.flink.streaming.runtime.tasks.StreamTask task,
/* 12 */            org.apache.flink.streaming.api.graph.StreamConfig config,
/* 13 */            org.apache.flink.streaming.api.operators.Output output,
/* 14 */            org.apache.flink.streaming.runtime.tasks.ProcessingTimeService processingTimeService) throws Exception {
/* 15 */          this.references = references;
/* 16 */          converter$3 = (((org.apache.flink.table.data.util.DataFormatConverters.StringConverter) references[0]));
/* 17 */          this.setup(task, config, output);
/* 18 */          if (this instanceof org.apache.flink.streaming.api.operators.AbstractStreamOperator) {
/* 19 */            ((org.apache.flink.streaming.api.operators.AbstractStreamOperator) this)
/* 20 */              .setProcessingTimeService(processingTimeService);
/* 21 */          }
/* 22 */        }
/* 23 */
/* 24 */        @Override
/* 25 */        public void open() throws Exception {
/* 26 */          super.open();
/* 27 */          
/* 28 */        }
/* 29 */
/* 30 */        @Override
/* 31 */        public void processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord element) throws Exception {
/* 32 */          org.apache.flink.table.data.RowData in1 = (org.apache.flink.table.data.RowData) element.getValue();
/* 33 */          
/* 34 */          
/* 35 */          
/* 36 */          
/* 37 */          
/* 38 */          output.collect(outElement.replace((java.lang.String) converter$3.toExternal((org.apache.flink.table.data.binary.BinaryStringData) in1)));
/* 39 */          
/* 40 */        }
/* 41 */
/* 42 */        
/* 43 */
/* 44 */        @Override
/* 45 */        public void close() throws Exception {
/* 46 */           super.close();
/* 47 */          
/* 48 */        }
/* 49 */
/* 50 */        
/* 51 */      }
/* 52 */    

19:53:58,610 WARN  org.apache.flink.runtime.taskmanager.Task                     - Source: Custom Source -> SourceConversion(table=[default_catalog.default_database.baselog], fields=[timestamp, zone_id,......]) -> Calc(select=[zone_id]) -> SinkConversionToString -> Sink: Print to Std. Out (2/16) (8e0da69492bc45fffdbc17cb32c8f99b) switched from RUNNING to FAILED.
java.lang.RuntimeException: Could not instantiate generated class 'SinkConversion$4'
	at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:67)
	at org.apache.flink.table.runtime.operators.CodeGenOperatorFactory.createStreamOperator(CodeGenOperatorFactory.java:40)
	at org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil.createOperator(StreamOperatorFactoryUtil.java:70)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:470)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:459)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:459)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:155)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:453)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:522)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkRuntimeException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68)
	at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78)
	at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:65)
	... 14 more
Caused by: org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739)
	at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66)
	... 16 more
Caused by: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81)
	at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197)
	... 19 more
Caused by: org.codehaus.commons.compiler.CompileException: Line 38, Column 144: Cannot cast "org.apache.flink.table.data.RowData" to "org.apache.flink.table.data.binary.BinaryStringData"
	at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12124)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5049)
	at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4416)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$Cast.accept(Java.java:4887)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5019)
	at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4416)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$Cast.accept(Java.java:4887)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3781)
	at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3760)
	at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3732)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3732)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2360)
	at org.codehaus.janino.UnitCompiler.access$1800(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1494)
	at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1487)
	at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2871)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487)
	at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1567)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3388)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432)
	at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411)
	at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406)
	at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406)
	at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237)
	at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207)
	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.runtime.generated.CompileUtils.doCompile(CompileUtils.java:78)
	... 25 more
19:53:58,611 WARN  org.apache.flink.runtime.taskmanager.Task                     - Source: Custom Source -> SourceConversion(table=[default_catalog.default_database.baselog], fields=[timestamp, zone_id,......]) -> Calc(select=[zone_id]) -> SinkConversionToString -> Sink: Print to Std. Out (13/16) (8941d6dc02abd60317416c8da46f0027) switched from RUNNING to FAILED.
java.lang.RuntimeException: Could not instantiate generated class 'SinkConversion$4'
	at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:67)
	at org.apache.flink.table.runtime.operators.CodeGenOperatorFactory.createStreamOperator(CodeGenOperatorFactory.java:40)
	at org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil.createOperator(StreamOperatorFactoryUtil.java:70)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:470)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:459)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:459)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:393)
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:155)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:453)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:522)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkRuntimeException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68)
	at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78)
	at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:65)
	... 14 more
Caused by: org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739)
	at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66)
	... 16 more
Caused by: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
	at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81)
	at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282)
	at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197)
	... 19 more
Caused by: org.codehaus.commons.compiler.CompileException: Line 38, Column 144: Cannot cast "org.apache.flink.table.data.RowData" to "org.apache.flink.table.data.binary.BinaryStringData"
	at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12124)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5049)
	at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4416)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$Cast.accept(Java.java:4887)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5019)
	at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4416)
	at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$Cast.accept(Java.java:4887)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5165)
	at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
	at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
	at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3781)
	at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3760)
	at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3732)
	at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3732)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2360)
	at org.codehaus.janino.UnitCompiler.access$1800(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1494)
	at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1487)
	at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2871)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487)
	at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1567)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3388)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432)
	at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215)
	at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411)
	at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406)
	at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406)
	at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237)
	at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216)
	at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207)
	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.runtime.generated.CompileUtils.doCompile(CompileUtils.java:78)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)