Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-28693

Codegen failed if the watermark is defined on a columnByExpression

    XMLWordPrintableJSON

Details

    Description

      The following code will throw an exception:

       

      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:

      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();
          }
      }

      What does the code do?

      1. read a stream from Kakfa
      2. create a derived column using an UDF expression
      3. define the watermark based on the derived column

      The full callstack:

       

      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 

      The generated class:

      /* 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 */    

      Addtional information:

      1. This is a regression from 1.14. The issue doesn't happen in 1.14
      2. It needs to be run in a flink cluster. It doesn't happen in a LocalEnvironment.
      3. It doen't happen when using `datagen` source instead of Kafka.

      Reproduce steps:

      1. download flink-1.15.1-bin-scala_2.12.tgz and run bin/start-cluster
      2. clone the test code from `git clone -b FLINK-28693 https://github.com/liuhb86/flink-test.git`
      3. run `gradlew shadowJar` to generate the jar files with the UDF function.
      4. run `gradlew test`. The exception occurs in about 10 seconds.

       

       

       

      Attachments

        Issue Links

          Activity

            People

              xuyangzhong xuyang
              liuhb86 Hongbo
              Votes:
              1 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: