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

Sergey Nuyanzin reassigned FLINK-28693:
---------------------------------------

    Assignee: xuyang

> Codegen failed if the watermark is defined on a columnByExpression
> ------------------------------------------------------------------
>
>                 Key: FLINK-28693
>                 URL: https://issues.apache.org/jira/browse/FLINK-28693
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Runtime
>    Affects Versions: 1.15.1
>            Reporter: Hongbo
>            Assignee: xuyang
>            Priority: Major
>              Labels: pull-request-available
>
> The following code will throw an exception:
>  
> {code:java}
> Table program cannot be compiled. This is a bug. Please file an issue.
>  ...
>  Caused by: org.codehaus.commons.compiler.CompileException: Line 29, Column 
> 54: Cannot determine simple type name "org" {code}
> {color:#000000}Code:{color}
> {code:java}
> public class TestUdf extends  ScalarFunction {
>     @DataTypeHint("TIMESTAMP(3)")
>     public LocalDateTime eval(String strDate) {
>        return LocalDateTime.now();
>     }
> }
> public class FlinkTest {
>     @Test
>     void testUdf() throws Exception {
>         //var env = StreamExecutionEnvironment.createLocalEnvironment();
>         // run `gradlew shadowJar` first to generate the uber jar.
>         // It contains the kafka connector and a dummy UDF function.
>         var env = 
> StreamExecutionEnvironment.createRemoteEnvironment("localhost", 8081,
>                 "build/libs/flink-test-all.jar");
>         env.setParallelism(1);
>         var tableEnv = StreamTableEnvironment.create(env);
>         tableEnv.createTemporarySystemFunction("TEST_UDF", TestUdf.class);
>         var testTable = tableEnv.from(TableDescriptor.forConnector("kafka")
>                 .schema(Schema.newBuilder()
>                         .column("time_stamp", DataTypes.STRING())
>                         .columnByExpression("udf_ts", "TEST_UDF(time_stamp)")
>                         .watermark("udf_ts", "udf_ts - INTERVAL '1' second")
>                         .build())
>                 // the kafka server doesn't need to exist. It fails in the 
> compile stage before fetching data.
>                 .option("properties.bootstrap.servers", "localhost:9092")
>                 .option("topic", "test_topic")
>                 .option("format", "json")
>                 .option("scan.startup.mode", "latest-offset")
>                 .build());
>         testTable.printSchema();
>         tableEnv.createTemporaryView("test", testTable );
>         var query = tableEnv.sqlQuery("select * from test");
>         var tableResult = 
> query.executeInsert(TableDescriptor.forConnector("print").build());
>         tableResult.await();
>     }
> }{code}
> What does the code do?
>  # read a stream from Kakfa
>  # create a derived column using an UDF expression
>  # define the watermark based on the derived column
> The full callstack:
>  
> {code:java}
> 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:94)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:97)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:68)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.GeneratedWatermarkGeneratorSupplier.createWatermarkGenerator(GeneratedWatermarkGeneratorSupplier.java:62)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.api.operators.source.ProgressiveTimestampsAndWatermarks.createMainOutput(ProgressiveTimestampsAndWatermarks.java:104)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.initializeMainOutput(SourceOperator.java:426)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.emitNextNotReading(SourceOperator.java:402)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:387)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:519)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:804)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:753)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
>  [flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927) 
> [flink-dist-1.15.1.jar:1.15.1]
>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741) 
> [flink-dist-1.15.1.jar:1.15.1]
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563) 
> [flink-dist-1.15.1.jar:1.15.1]
>     at java.lang.Thread.run(Thread.java:829) [?:?]
> Caused by: 
> org.apache.flink.shaded.guava30.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.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2051)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     ... 18 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:107)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$0(CompileUtils.java:92)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4864)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     ... 18 more
> Caused by: org.codehaus.commons.compiler.CompileException: Line 29, Column 
> 54: Cannot determine simple type name "org"
>     at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6833) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6594) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:7121) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$17000(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6529)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6490)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$NewClassInstance.accept(Java.java:5190) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6490) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6469) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$Rvalue.accept(Java.java:4116) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9237)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9123) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9025) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5062) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4423)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4396)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4396) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5662) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3783) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3762)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3734)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3734) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2360) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$1800(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1494)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1487)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2874) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1567) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3388) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75) 
> ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:104)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$0(CompileUtils.java:92)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4864)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859)
>  ~[flink-dist-1.15.1.jar:1.15.1]
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92)
>  ~[flink-table-runtime-1.15.1.jar:1.15.1]
>     ... 18 more
> 2022-07-26 09:53:12,770 INFO  
> org.apache.flink.connector.base.source.reader.SourceReaderBase [] - Closing 
> Source Reader.
> 2022-07-26 09:53:12,772 WARN  org.apache.flink.runtime.taskmanager.Task       
>              [] - Source: *anonymous_kafka$1*[1] -> Calc[2] -> Sink: 
> *anonymous_print$2*[3] (1/1)#0 (5283ea1dc807419920579ac3f255a090) switched 
> from RUNNING to FAILED with failure cause: java.lang.RuntimeException: Could 
> not instantiate generated class 'WatermarkGenerator$0'
>     at 
> org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:74)
>     at 
> org.apache.flink.table.runtime.generated.GeneratedWatermarkGeneratorSupplier.createWatermarkGenerator(GeneratedWatermarkGeneratorSupplier.java:62)
>     at 
> org.apache.flink.streaming.api.operators.source.ProgressiveTimestampsAndWatermarks.createMainOutput(ProgressiveTimestampsAndWatermarks.java:104)
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.initializeMainOutput(SourceOperator.java:426)
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.emitNextNotReading(SourceOperator.java:402)
>     at 
> org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:387)
>     at 
> org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
>     at 
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:519)
>     at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:804)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:753)
>     at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
>     at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927)
>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
>     at java.base/java.lang.Thread.run(Thread.java:829)
> 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:94)
>     at 
> org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:101)
>     at 
> org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:68)
>     ... 16 more
> Caused by: 
> org.apache.flink.shaded.guava30.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.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2051)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859)
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92)
>     ... 18 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:107)
>     at 
> org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$0(CompileUtils.java:92)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4864)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155)
>     at 
> org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045)
>     ... 21 more
> Caused by: org.codehaus.commons.compiler.CompileException: Line 29, Column 
> 54: Cannot determine simple type name "org"
>     at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6833)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6594)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at 
> org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607)
>     at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573)
>     at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215)
>     at 
> org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481)
>     at 
> org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476)
>     at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928)
>     at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476)
>     at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469)
>     at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927)
>     at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469)
>     at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:7121)
>     at org.codehaus.janino.UnitCompiler.access$17000(UnitCompiler.java:215)
>     at 
> org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6529)
>     at 
> org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6490)
>     at org.codehaus.janino.Java$NewClassInstance.accept(Java.java:5190)
>     at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6490)
>     at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6469)
>     at org.codehaus.janino.Java$Rvalue.accept(Java.java:4116)
>     at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469)
>     at 
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9237)
>     at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9123)
>     at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9025)
>     at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5062)
>     at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
>     at 
> org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4423)
>     at 
> org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4396)
>     at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073)
>     at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4396)
>     at 
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5662)
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3783)
>     at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215)
>     at 
> org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3762)
>     at 
> org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3734)
>     at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073)
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3734)
>     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:2874)
>     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:104)
>     ... 27 more {code}
> The generated class:
> {code:java}
> /* 1 */
> /* 2 */      public final class WatermarkGenerator$0
> /* 3 */          extends 
> org.apache.flink.table.runtime.generated.WatermarkGenerator {
> /* 4 */
> /* 5 */        private transient 
> org.apache.flink.table.runtime.typeutils.StringDataSerializer 
> typeSerializer$2;
> /* 6 */        private transient com.flinktest.TestUdf 
> function_com$flinktest$TestUdf;
> /* 7 */        private transient 
> org.apache.flink.table.data.conversion.StringStringConverter converter$4;
> /* 8 */        private transient 
> org.apache.flink.table.data.conversion.TimestampLocalDateTimeConverter 
> converter$6;
> /* 9 */        
> /* 10 */        private transient 
> org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context
> /* 11 */        context;
> /* 12 */        
> /* 13 */
> /* 14 */        public WatermarkGenerator$0(Object[] references) throws 
> Exception {
> /* 15 */          typeSerializer$2 = 
> (((org.apache.flink.table.runtime.typeutils.StringDataSerializer) 
> references[0]));
> /* 16 */          function_com$flinktest$TestUdf = (((com.flinktest.TestUdf) 
> references[1]));
> /* 17 */          converter$4 = 
> (((org.apache.flink.table.data.conversion.StringStringConverter) 
> references[2]));
> /* 18 */          converter$6 = 
> (((org.apache.flink.table.data.conversion.TimestampLocalDateTimeConverter) 
> references[3]));
> /* 19 */          
> /* 20 */          int len = references.length;
> /* 21 */          context =
> /* 22 */          
> (org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context) 
> references[len-1];
> /* 23 */          
> /* 24 */        }
> /* 25 */
> /* 26 */        @Override
> /* 27 */        public void open(org.apache.flink.configuration.Configuration 
> parameters) throws Exception {
> /* 28 */          
> /* 29 */          function_com$flinktest$TestUdf.open(new 
> org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGeneratorFunctionContextWrapper(context));
> /* 30 */                 
> /* 31 */          
> /* 32 */          converter$4.open(this.getClass().getClassLoader());
> /* 33 */                     
> /* 34 */          
> /* 35 */          converter$6.open(this.getClass().getClassLoader());
> /* 36 */                     
> /* 37 */        }
> /* 38 */
> /* 39 */        @Override
> /* 40 */        public Long 
> currentWatermark(org.apache.flink.table.data.RowData row) throws Exception {
> /* 41 */          
> /* 42 */          org.apache.flink.table.data.binary.BinaryStringData field$1;
> /* 43 */          boolean isNull$1;
> /* 44 */          org.apache.flink.table.data.binary.BinaryStringData field$3;
> /* 45 */          java.time.LocalDateTime externalResult$5;
> /* 46 */          org.apache.flink.table.data.TimestampData result$7;
> /* 47 */          boolean isNull$7;
> /* 48 */          boolean isNull$8;
> /* 49 */          org.apache.flink.table.data.TimestampData result$9;
> /* 50 */          
> /* 51 */          isNull$1 = row.isNullAt(0);
> /* 52 */          field$1 = 
> org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 53 */          if (!isNull$1) {
> /* 54 */            field$1 = 
> ((org.apache.flink.table.data.binary.BinaryStringData) row.getString(0));
> /* 55 */          }
> /* 56 */          field$3 = field$1;
> /* 57 */          if (!isNull$1) {
> /* 58 */            field$3 = 
> (org.apache.flink.table.data.binary.BinaryStringData) 
> (typeSerializer$2.copy(field$3));
> /* 59 */          }
> /* 60 */                  
> /* 61 */          
> /* 62 */          
> /* 63 */          
> /* 64 */          
> /* 65 */          externalResult$5 = (java.time.LocalDateTime) 
> function_com$flinktest$TestUdf
> /* 66 */            .eval(isNull$1 ? null : ((java.lang.String) 
> converter$4.toExternal((org.apache.flink.table.data.binary.BinaryStringData) 
> field$3)));
> /* 67 */          
> /* 68 */          isNull$7 = externalResult$5 == null;
> /* 69 */          result$7 = null;
> /* 70 */          if (!isNull$7) {
> /* 71 */            result$7 = (org.apache.flink.table.data.TimestampData) 
> converter$6.toInternalOrNull((java.time.LocalDateTime) externalResult$5);
> /* 72 */          }
> /* 73 */          
> /* 74 */          
> /* 75 */          isNull$8 = isNull$7 || false;
> /* 76 */          result$9 = null;
> /* 77 */          if (!isNull$8) {
> /* 78 */            
> /* 79 */          
> /* 80 */          result$9 = 
> org.apache.flink.table.data.TimestampData.fromEpochMillis(result$7.getMillisecond()
>  - ((long) 1000L), result$7.getNanoOfMillisecond());
> /* 81 */          
> /* 82 */            
> /* 83 */          }
> /* 84 */          
> /* 85 */          if (isNull$8) {
> /* 86 */            return null;
> /* 87 */          } else {
> /* 88 */            return result$9.getMillisecond();
> /* 89 */          }
> /* 90 */        }
> /* 91 */
> /* 92 */        @Override
> /* 93 */        public void close() throws Exception {
> /* 94 */          
> /* 95 */          function_com$flinktest$TestUdf.close();
> /* 96 */                 
> /* 97 */        }
> /* 98 */      }
> /* 99 */    /* 1 */
> /* 2 */      public final class WatermarkGenerator$0
> /* 3 */          extends 
> org.apache.flink.table.runtime.generated.WatermarkGenerator {
> /* 4 */
> /* 5 */        private transient 
> org.apache.flink.table.runtime.typeutils.StringDataSerializer 
> typeSerializer$2;
> /* 6 */        private transient com.flinktest.TestUdf 
> function_com$flinktest$TestUdf;
> /* 7 */        private transient 
> org.apache.flink.table.data.conversion.StringStringConverter converter$4;
> /* 8 */        private transient 
> org.apache.flink.table.data.conversion.TimestampLocalDateTimeConverter 
> converter$6;
> /* 9 */        
> /* 10 */        private transient 
> org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context
> /* 11 */        context;
> /* 12 */        
> /* 13 */
> /* 14 */        public WatermarkGenerator$0(Object[] references) throws 
> Exception {
> /* 15 */          typeSerializer$2 = 
> (((org.apache.flink.table.runtime.typeutils.StringDataSerializer) 
> references[0]));
> /* 16 */          function_com$flinktest$TestUdf = (((com.flinktest.TestUdf) 
> references[1]));
> /* 17 */          converter$4 = 
> (((org.apache.flink.table.data.conversion.StringStringConverter) 
> references[2]));
> /* 18 */          converter$6 = 
> (((org.apache.flink.table.data.conversion.TimestampLocalDateTimeConverter) 
> references[3]));
> /* 19 */          
> /* 20 */          int len = references.length;
> /* 21 */          context =
> /* 22 */          
> (org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context) 
> references[len-1];
> /* 23 */          
> /* 24 */        }
> /* 25 */
> /* 26 */        @Override
> /* 27 */        public void open(org.apache.flink.configuration.Configuration 
> parameters) throws Exception {
> /* 28 */          
> /* 29 */          function_com$flinktest$TestUdf.open(new 
> org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGeneratorFunctionContextWrapper(context));
> /* 30 */                 
> /* 31 */          
> /* 32 */          converter$4.open(this.getClass().getClassLoader());
> /* 33 */                     
> /* 34 */          
> /* 35 */          converter$6.open(this.getClass().getClassLoader());
> /* 36 */                     
> /* 37 */        }
> /* 38 */
> /* 39 */        @Override
> /* 40 */        public Long 
> currentWatermark(org.apache.flink.table.data.RowData row) throws Exception {
> /* 41 */          
> /* 42 */          org.apache.flink.table.data.binary.BinaryStringData field$1;
> /* 43 */          boolean isNull$1;
> /* 44 */          org.apache.flink.table.data.binary.BinaryStringData field$3;
> /* 45 */          java.time.LocalDateTime externalResult$5;
> /* 46 */          org.apache.flink.table.data.TimestampData result$7;
> /* 47 */          boolean isNull$7;
> /* 48 */          boolean isNull$8;
> /* 49 */          org.apache.flink.table.data.TimestampData result$9;
> /* 50 */          
> /* 51 */          isNull$1 = row.isNullAt(0);
> /* 52 */          field$1 = 
> org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 53 */          if (!isNull$1) {
> /* 54 */            field$1 = 
> ((org.apache.flink.table.data.binary.BinaryStringData) row.getString(0));
> /* 55 */          }
> /* 56 */          field$3 = field$1;
> /* 57 */          if (!isNull$1) {
> /* 58 */            field$3 = 
> (org.apache.flink.table.data.binary.BinaryStringData) 
> (typeSerializer$2.copy(field$3));
> /* 59 */          }
> /* 60 */                  
> /* 61 */          
> /* 62 */          
> /* 63 */          
> /* 64 */          
> /* 65 */          externalResult$5 = (java.time.LocalDateTime) 
> function_com$flinktest$TestUdf
> /* 66 */            .eval(isNull$1 ? null : ((java.lang.String) 
> converter$4.toExternal((org.apache.flink.table.data.binary.BinaryStringData) 
> field$3)));
> /* 67 */          
> /* 68 */          isNull$7 = externalResult$5 == null;
> /* 69 */          result$7 = null;
> /* 70 */          if (!isNull$7) {
> /* 71 */            result$7 = (org.apache.flink.table.data.TimestampData) 
> converter$6.toInternalOrNull((java.time.LocalDateTime) externalResult$5);
> /* 72 */          }
> /* 73 */          
> /* 74 */          
> /* 75 */          isNull$8 = isNull$7 || false;
> /* 76 */          result$9 = null;
> /* 77 */          if (!isNull$8) {
> /* 78 */            
> /* 79 */          
> /* 80 */          result$9 = 
> org.apache.flink.table.data.TimestampData.fromEpochMillis(result$7.getMillisecond()
>  - ((long) 1000L), result$7.getNanoOfMillisecond());
> /* 81 */          
> /* 82 */            
> /* 83 */          }
> /* 84 */          
> /* 85 */          if (isNull$8) {
> /* 86 */            return null;
> /* 87 */          } else {
> /* 88 */            return result$9.getMillisecond();
> /* 89 */          }
> /* 90 */        }
> /* 91 */
> /* 92 */        @Override
> /* 93 */        public void close() throws Exception {
> /* 94 */          
> /* 95 */          function_com$flinktest$TestUdf.close();
> /* 96 */                 
> /* 97 */        }
> /* 98 */      }
> /* 99 */    {code}
> Addtional information:
>  # This is a regression from 1.14. The issue doesn't happen in 1.14
>  # It needs to be run in a flink cluster. It doesn't happen in a 
> LocalEnvironment.
>  # It doen't happen when using `datagen` source instead of Kafka.
> Reproduce steps:
>  # download flink-1.15.1-bin-scala_2.12.tgz and run bin/start-cluster
>  # clone the test code from `git clone -b FLINK-28693 
> [https://github.com/liuhb86/flink-test.git]`
>  # run `gradlew shadowJar` to generate the jar files with the UDF function.
>  # run `gradlew test`. The exception occurs in about 10 seconds.
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to