You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Yun Gao (Jira)" <ji...@apache.org> on 2022/04/13 06:28:04 UTC
[jira] [Updated] (FLINK-22742) Lookup join condition with process time throws org.codehaus.commons.compiler.CompileException
[ https://issues.apache.org/jira/browse/FLINK-22742?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Yun Gao updated FLINK-22742:
----------------------------
Fix Version/s: 1.16.0
> Lookup join condition with process time throws org.codehaus.commons.compiler.CompileException
> ---------------------------------------------------------------------------------------------
>
> Key: FLINK-22742
> URL: https://issues.apache.org/jira/browse/FLINK-22742
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Runtime
> Affects Versions: 1.12.0, 1.13.0, 1.14.0
> Reporter: Caizhi Weng
> Priority: Minor
> Labels: auto-deprioritized-major
> Fix For: 1.15.0, 1.16.0
>
>
> Add the following test case to {{org.apache.flink.table.api.TableEnvironmentITCase}} to reproduce this bug.
> {code:scala}
> @Test
> def myTest(): Unit = {
> val id1 = TestValuesTableFactory.registerData(
> Seq(Row.of("abc", LocalDateTime.of(2000, 1, 1, 0, 0))))
> val ddl1 =
> s"""
> |CREATE TABLE Ta (
> | id VARCHAR,
> | ts TIMESTAMP,
> | proc AS PROCTIME()
> |) WITH (
> | 'connector' = 'values',
> | 'data-id' = '$id1',
> | 'bounded' = 'true'
> |)
> |""".stripMargin
> tEnv.executeSql(ddl1)
> val id2 = TestValuesTableFactory.registerData(
> Seq(Row.of("abc", LocalDateTime.of(2000, 1, 2, 0, 0))))
> val ddl2 =
> s"""
> |CREATE TABLE Tb (
> | id VARCHAR,
> | ts TIMESTAMP
> |) WITH (
> | 'connector' = 'values',
> | 'data-id' = '$id2',
> | 'bounded' = 'true'
> |)
> |""".stripMargin
> tEnv.executeSql(ddl2)
> val it = tEnv.executeSql(
> """
> |SELECT * FROM Ta AS t1
> |INNER JOIN Tb FOR SYSTEM_TIME AS OF t1.proc AS t2
> |ON t1.id = t2.id
> |WHERE CAST(coalesce(t1.ts, t2.ts) AS VARCHAR) >= CONCAT(DATE_FORMAT(t1.proc, 'yyyy-MM-dd'), ' 00:00:00')
> |""".stripMargin).collect()
> while (it.hasNext) {
> System.out.println(it.next())
> }
> }
> {code}
> The exception stack is
> {code}
> /* 1 */
> /* 2 */ public class JoinTableFuncCollector$25 extends org.apache.flink.table.runtime.collector.TableFunctionCollector {
> /* 3 */
> /* 4 */ org.apache.flink.table.data.GenericRowData out = new org.apache.flink.table.data.GenericRowData(2);
> /* 5 */org.apache.flink.table.data.utils.JoinedRowData joinedRow$9 = new org.apache.flink.table.data.utils.JoinedRowData();
> /* 6 */
> /* 7 */private final org.apache.flink.table.data.binary.BinaryStringData str$17 = org.apache.flink.table.data.binary.BinaryStringData.fromString("yyyy-MM-dd");
> /* 8 */
> /* 9 */private static final java.util.TimeZone timeZone =
> /* 10 */ java.util.TimeZone.getTimeZone("Asia/Shanghai");
> /* 11 */
> /* 12 */private final org.apache.flink.table.data.binary.BinaryStringData str$20 = org.apache.flink.table.data.binary.BinaryStringData.fromString(" 00:00:00");
> /* 13 */
> /* 14 */
> /* 15 */ public JoinTableFuncCollector$25(Object[] references) throws Exception {
> /* 16 */
> /* 17 */ }
> /* 18 */
> /* 19 */ @Override
> /* 20 */ public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
> /* 21 */
> /* 22 */ }
> /* 23 */
> /* 24 */ @Override
> /* 25 */ public void collect(Object record) throws Exception {
> /* 26 */ org.apache.flink.table.data.RowData in1 = (org.apache.flink.table.data.RowData) getInput();
> /* 27 */ org.apache.flink.table.data.RowData in2 = (org.apache.flink.table.data.RowData) record;
> /* 28 */
> /* 29 */ org.apache.flink.table.data.binary.BinaryStringData field$7;
> /* 30 */boolean isNull$7;
> /* 31 */org.apache.flink.table.data.TimestampData field$8;
> /* 32 */boolean isNull$8;
> /* 33 */org.apache.flink.table.data.TimestampData field$10;
> /* 34 */boolean isNull$10;
> /* 35 */boolean isNull$13;
> /* 36 */org.apache.flink.table.data.binary.BinaryStringData result$14;
> /* 37 */org.apache.flink.table.data.TimestampData field$15;
> /* 38 */boolean isNull$15;
> /* 39 */org.apache.flink.table.data.TimestampData result$16;
> /* 40 */boolean isNull$18;
> /* 41 */org.apache.flink.table.data.binary.BinaryStringData result$19;
> /* 42 */boolean isNull$21;
> /* 43 */org.apache.flink.table.data.binary.BinaryStringData result$22;
> /* 44 */boolean isNull$23;
> /* 45 */boolean result$24;
> /* 46 */ isNull$15 = in1.isNullAt(2);
> /* 47 */field$15 = null;
> /* 48 */if (!isNull$15) {
> /* 49 */ field$15 = in1.getTimestamp(2, 3);
> /* 50 */}
> /* 51 */isNull$8 = in2.isNullAt(1);
> /* 52 */field$8 = null;
> /* 53 */if (!isNull$8) {
> /* 54 */ field$8 = in2.getTimestamp(1, 6);
> /* 55 */}
> /* 56 */isNull$7 = in2.isNullAt(0);
> /* 57 */field$7 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 58 */if (!isNull$7) {
> /* 59 */ field$7 = ((org.apache.flink.table.data.binary.BinaryStringData) in2.getString(0));
> /* 60 */}
> /* 61 */isNull$10 = in1.isNullAt(1);
> /* 62 */field$10 = null;
> /* 63 */if (!isNull$10) {
> /* 64 */ field$10 = in1.getTimestamp(1, 6);
> /* 65 */}
> /* 66 */
> /* 67 */
> /* 68 */
> /* 69 */boolean result$11 = !isNull$10;
> /* 70 */org.apache.flink.table.data.TimestampData result$12 = null;
> /* 71 */boolean isNull$12;
> /* 72 */if (result$11) {
> /* 73 */
> /* 74 */ isNull$12 = isNull$10;
> /* 75 */ if (!isNull$12) {
> /* 76 */ result$12 = field$10;
> /* 77 */ }
> /* 78 */}
> /* 79 */else {
> /* 80 */
> /* 81 */ isNull$12 = isNull$8;
> /* 82 */ if (!isNull$12) {
> /* 83 */ result$12 = field$8;
> /* 84 */ }
> /* 85 */}
> /* 86 */isNull$13 = isNull$12;
> /* 87 */result$14 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 88 */if (!isNull$13) {
> /* 89 */
> /* 90 */ result$14 = org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$12, 6));
> /* 91 */ isNull$13 = (result$14 == null);
> /* 92 */}
> /* 93 */
> /* 94 */
> /* 95 */
> /* 96 */result$16 = org.apache.flink.table.data.TimestampData.fromEpochMillis(
> /* 97 */ ctx.timerService().currentProcessingTime());
> /* 98 */
> /* 99 */isNull$18 = false || false;
> /* 100 */result$19 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 101 */if (!isNull$18) {
> /* 102 */
> /* 103 */ result$19 = org.apache.flink.table.data.binary.BinaryStringData.fromString(
> /* 104 */org.apache.flink.table.runtime.functions.SqlDateTimeUtils.dateFormat(result$16, ((org.apache.flink.table.data.binary.BinaryStringData) str$17).toString(), timeZone)
> /* 105 */ );
> /* 106 */ isNull$18 = (result$19 == null);
> /* 107 */}
> /* 108 */
> /* 109 */
> /* 110 */result$22 = org.apache.flink.table.data.binary.BinaryStringDataUtil.concat(( isNull$18 ) ? null : (result$19), ( false ) ? null : (((org.apache.flink.table.data.binary.BinaryStringData) str$20)));
> /* 111 */isNull$21 = (result$22 == null);
> /* 112 */if (isNull$21) {
> /* 113 */ result$22 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
> /* 114 */}
> /* 115 */
> /* 116 */isNull$23 = isNull$13 || isNull$21;
> /* 117 */result$24 = false;
> /* 118 */if (!isNull$23) {
> /* 119 */
> /* 120 */ result$24 = ((result$14 == null) ? ((result$22 == null) ? 0 : -1) : ((result$22 == null) ? 1 : (result$14.compareTo(result$22)))) >= 0;
> /* 121 */
> /* 122 */}
> /* 123 */
> /* 124 */if (result$24) {
> /* 125 */
> /* 126 */
> /* 127 */
> /* 128 */
> /* 129 */
> /* 130 */if (isNull$7) {
> /* 131 */ out.setField(0, null);
> /* 132 */} else {
> /* 133 */ out.setField(0, field$7);
> /* 134 */}
> /* 135 */
> /* 136 */
> /* 137 */
> /* 138 */if (isNull$8) {
> /* 139 */ out.setField(1, null);
> /* 140 */} else {
> /* 141 */ out.setField(1, field$8);
> /* 142 */}
> /* 143 */
> /* 144 */
> /* 145 */joinedRow$9.replace(in1, out);
> /* 146 */joinedRow$9.setRowKind(in1.getRowKind());
> /* 147 */outputResult(joinedRow$9);
> /* 148 */
> /* 149 */}
> /* 150 */
> /* 151 */ }
> /* 152 */
> /* 153 */ @Override
> /* 154 */ public void close() throws Exception {
> /* 155 */
> /* 156 */ }
> /* 157 */ }
> /* 158 */
> java.lang.RuntimeException: Failed to fetch next result
> at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:109)
> at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.hasNext(CollectResultIterator.java:80)
> at org.apache.flink.table.api.internal.TableResultImpl$CloseableRowIteratorWrapper.hasNext(TableResultImpl.java:370)
> at org.apache.flink.table.api.TableEnvironmentITCase.myTest(TableEnvironmentITCase.scala:129)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
> at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
> at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> at org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:239)
> at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
> at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
> at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
> at org.junit.rules.RunRules.evaluate(RunRules.java:20)
> at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
> at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
> at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at org.junit.runners.Suite.runChild(Suite.java:128)
> at org.junit.runners.Suite.runChild(Suite.java:27)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
> at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
> at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
> at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230)
> at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58)
> Caused by: java.io.IOException: Failed to fetch job execution result
> at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:177)
> at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.next(CollectResultFetcher.java:120)
> at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:106)
> ... 39 more
> Caused by: java.util.concurrent.ExecutionException: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
> at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
> at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:175)
> ... 41 more
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144)
> at org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$3(MiniClusterJobClient.java:137)
> at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
> at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614)
> at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983)
> at org.apache.flink.runtime.minicluster.MiniClusterJobClient.getJobExecutionResult(MiniClusterJobClient.java:134)
> at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:174)
> ... 41 more
> Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
> at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138)
> at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82)
> at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:207)
> at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:197)
> at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:188)
> at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:677)
> at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
> at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:435)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305)
> at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212)
> at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77)
> at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
> at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
> at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
> at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
> at akka.actor.ActorCell.invoke(ActorCell.scala:561)
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
> at akka.dispatch.Mailbox.run(Mailbox.scala:225)
> at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: java.lang.RuntimeException: Could not instantiate generated class 'JoinTableFuncCollector$25'
> at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:59)
> at org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner.open(LookupJoinRunner.java:63)
> at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:34)
> at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:102)
> at org.apache.flink.streaming.api.operators.ProcessOperator.open(ProcessOperator.java:56)
> at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:437)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:582)
> at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.call(StreamTaskActionExecutor.java:100)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.executeRestore(StreamTask.java:562)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:537)
> at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:759)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
> 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:76)
> at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:80)
> at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:53)
> ... 13 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:74)
> ... 15 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:89)
> at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:74)
> 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)
> ... 18 more
> Caused by: org.codehaus.commons.compiler.CompileException: Line 97, Column 6: Unknown variable or type "ctx"
> at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211)
> at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6860)
> at org.codehaus.janino.UnitCompiler.access$13600(UnitCompiler.java:215)
> at org.codehaus.janino.UnitCompiler$22.visitPackage(UnitCompiler.java:6472)
> at org.codehaus.janino.UnitCompiler$22.visitPackage(UnitCompiler.java:6469)
> at org.codehaus.janino.Java$Package.accept(Java.java:4248)
> at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469)
> at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6855)
> at org.codehaus.janino.UnitCompiler.access$14200(UnitCompiler.java:215)
> at org.codehaus.janino.UnitCompiler$22$2$1.visitAmbiguousName(UnitCompiler.java:6497)
> at org.codehaus.janino.UnitCompiler$22$2$1.visitAmbiguousName(UnitCompiler.java:6494)
> at org.codehaus.janino.Java$AmbiguousName.accept(Java.java:4224)
> at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6494)
> at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6490)
> at org.codehaus.janino.Java$Lvalue.accept(Java.java:4148)
> 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.findIMethod(UnitCompiler.java:9026)
> at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:7106)
> at org.codehaus.janino.UnitCompiler.access$15800(UnitCompiler.java:215)
> at org.codehaus.janino.UnitCompiler$22$2.visitMethodInvocation(UnitCompiler.java:6517)
> at org.codehaus.janino.UnitCompiler$22$2.visitMethodInvocation(UnitCompiler.java:6490)
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073)
> 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.findIMethod(UnitCompiler.java:9026)
> at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:7106)
> at org.codehaus.janino.UnitCompiler.access$15800(UnitCompiler.java:215)
> at org.codehaus.janino.UnitCompiler$22$2.visitMethodInvocation(UnitCompiler.java:6517)
> at org.codehaus.janino.UnitCompiler$22$2.visitMethodInvocation(UnitCompiler.java:6490)
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073)
> 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:3792)
> at org.codehaus.janino.UnitCompiler.access$6100(UnitCompiler.java:215)
> at org.codehaus.janino.UnitCompiler$13.visitAssignment(UnitCompiler.java:3754)
> at org.codehaus.janino.UnitCompiler$13.visitAssignment(UnitCompiler.java:3734)
> at org.codehaus.janino.Java$Assignment.accept(Java.java:4477)
> 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:86)
> ... 24 more
> {code}
> This is because {{LookupJoinCodeGenerator#generateTableFunctionCollectorForJoinTable}} doesn't add a {{ctx}} member to its generated class.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)