Xingbo Huang created FLINK-29093: ------------------------------------ Summary: LookupJoinITCase failed with InternalCompilerException Key: FLINK-29093 URL: https://issues.apache.org/jira/browse/FLINK-29093 Project: Flink Issue Type: Bug Components: Table SQL / Runtime Affects Versions: 1.16.0 Reporter: Xingbo Huang Fix For: 1.16.0
{code:java} 2022-08-24T03:45:02.5915521Z Aug 24 03:45:02 org.apache.flink.runtime.client.JobExecutionException: Job execution failed. 2022-08-24T03:45:02.5916823Z Aug 24 03:45:02 at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144) 2022-08-24T03:45:02.5919320Z Aug 24 03:45:02 at org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$3(MiniClusterJobClient.java:141) 2022-08-24T03:45:02.5920833Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) 2022-08-24T03:45:02.5922361Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) 2022-08-24T03:45:02.5923733Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 2022-08-24T03:45:02.5924922Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) 2022-08-24T03:45:02.5926191Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$1(AkkaInvocationHandler.java:268) 2022-08-24T03:45:02.5927677Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) 2022-08-24T03:45:02.5929091Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) 2022-08-24T03:45:02.5930430Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 2022-08-24T03:45:02.5931966Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) 2022-08-24T03:45:02.5933293Z Aug 24 03:45:02 at org.apache.flink.util.concurrent.FutureUtils.doForward(FutureUtils.java:1277) 2022-08-24T03:45:02.5934708Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$null$1(ClassLoadingUtils.java:93) 2022-08-24T03:45:02.5936228Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68) 2022-08-24T03:45:02.5937998Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$guardCompletionWithContextClassLoader$2(ClassLoadingUtils.java:92) 2022-08-24T03:45:02.5939627Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) 2022-08-24T03:45:02.5941051Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) 2022-08-24T03:45:02.5942650Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) 2022-08-24T03:45:02.5944203Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) 2022-08-24T03:45:02.5945740Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$1.onComplete(AkkaFutureUtils.java:47) 2022-08-24T03:45:02.5947020Z Aug 24 03:45:02 at akka.dispatch.OnComplete.internal(Future.scala:300) 2022-08-24T03:45:02.5948130Z Aug 24 03:45:02 at akka.dispatch.OnComplete.internal(Future.scala:297) 2022-08-24T03:45:02.5949255Z Aug 24 03:45:02 at akka.dispatch.japi$CallbackBridge.apply(Future.scala:224) 2022-08-24T03:45:02.5950405Z Aug 24 03:45:02 at akka.dispatch.japi$CallbackBridge.apply(Future.scala:221) 2022-08-24T03:45:02.5951638Z Aug 24 03:45:02 at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60) 2022-08-24T03:45:02.5953564Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$DirectExecutionContext.execute(AkkaFutureUtils.java:65) 2022-08-24T03:45:02.5955214Z Aug 24 03:45:02 at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68) 2022-08-24T03:45:02.5956587Z Aug 24 03:45:02 at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284) 2022-08-24T03:45:02.5958037Z Aug 24 03:45:02 at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284) 2022-08-24T03:45:02.5959448Z Aug 24 03:45:02 at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284) 2022-08-24T03:45:02.5960679Z Aug 24 03:45:02 at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:621) 2022-08-24T03:45:02.5962183Z Aug 24 03:45:02 at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:24) 2022-08-24T03:45:02.5963694Z Aug 24 03:45:02 at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:23) 2022-08-24T03:45:02.5965024Z Aug 24 03:45:02 at scala.concurrent.Future.$anonfun$andThen$1(Future.scala:532) 2022-08-24T03:45:02.5966227Z Aug 24 03:45:02 at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:29) 2022-08-24T03:45:02.5967444Z Aug 24 03:45:02 at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:29) 2022-08-24T03:45:02.5968642Z Aug 24 03:45:02 at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60) 2022-08-24T03:45:02.5969913Z Aug 24 03:45:02 at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:63) 2022-08-24T03:45:02.5971340Z Aug 24 03:45:02 at akka.dispatch.BatchingExecutor$BlockableBatch.$anonfun$run$1(BatchingExecutor.scala:100) 2022-08-24T03:45:02.5972824Z Aug 24 03:45:02 at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12) 2022-08-24T03:45:02.5974100Z Aug 24 03:45:02 at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:81) 2022-08-24T03:45:02.5975389Z Aug 24 03:45:02 at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:100) 2022-08-24T03:45:02.5976670Z Aug 24 03:45:02 at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:49) 2022-08-24T03:45:02.5978063Z Aug 24 03:45:02 at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:48) 2022-08-24T03:45:02.5979479Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) 2022-08-24T03:45:02.5980728Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) 2022-08-24T03:45:02.5982143Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) 2022-08-24T03:45:02.5983410Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) 2022-08-24T03:45:02.5984613Z Aug 24 03:45:02 Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy 2022-08-24T03:45:02.5986005Z Aug 24 03:45:02 at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:139) 2022-08-24T03:45:02.5988034Z Aug 24 03:45:02 at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:83) 2022-08-24T03:45:02.5989611Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:256) 2022-08-24T03:45:02.5990833Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:247) 2022-08-24T03:45:02.5992309Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:240) 2022-08-24T03:45:02.5993616Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:738) 2022-08-24T03:45:02.5994883Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:715) 2022-08-24T03:45:02.5996320Z Aug 24 03:45:02 at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:78) 2022-08-24T03:45:02.5997572Z Aug 24 03:45:02 at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:477) 2022-08-24T03:45:02.5998837Z Aug 24 03:45:02 at sun.reflect.GeneratedMethodAccessor17.invoke(Unknown Source) 2022-08-24T03:45:02.6000095Z Aug 24 03:45:02 at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) 2022-08-24T03:45:02.6001334Z Aug 24 03:45:02 at java.lang.reflect.Method.invoke(Method.java:498) 2022-08-24T03:45:02.6002879Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:309) 2022-08-24T03:45:02.6004484Z Aug 24 03:45:02 at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) 2022-08-24T03:45:02.6006045Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:307) 2022-08-24T03:45:02.6007510Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:222) 2022-08-24T03:45:02.6009009Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:84) 2022-08-24T03:45:02.6010458Z Aug 24 03:45:02 at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:168) 2022-08-24T03:45:02.6011933Z Aug 24 03:45:02 at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) 2022-08-24T03:45:02.6013136Z Aug 24 03:45:02 at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) 2022-08-24T03:45:02.6014334Z Aug 24 03:45:02 at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) 2022-08-24T03:45:02.6015516Z Aug 24 03:45:02 at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) 2022-08-24T03:45:02.6016746Z Aug 24 03:45:02 at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) 2022-08-24T03:45:02.6017988Z Aug 24 03:45:02 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 2022-08-24T03:45:02.6019207Z Aug 24 03:45:02 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 2022-08-24T03:45:02.6020453Z Aug 24 03:45:02 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 2022-08-24T03:45:02.6021664Z Aug 24 03:45:02 at akka.actor.Actor.aroundReceive(Actor.scala:537) 2022-08-24T03:45:02.6022813Z Aug 24 03:45:02 at akka.actor.Actor.aroundReceive$(Actor.scala:535) 2022-08-24T03:45:02.6023947Z Aug 24 03:45:02 at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) 2022-08-24T03:45:02.6025133Z Aug 24 03:45:02 at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580) 2022-08-24T03:45:02.6026232Z Aug 24 03:45:02 at akka.actor.ActorCell.invoke(ActorCell.scala:548) 2022-08-24T03:45:02.6027190Z Aug 24 03:45:02 at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) 2022-08-24T03:45:02.6028082Z Aug 24 03:45:02 at akka.dispatch.Mailbox.run(Mailbox.scala:231) 2022-08-24T03:45:02.6029101Z Aug 24 03:45:02 at akka.dispatch.Mailbox.exec(Mailbox.scala:243) 2022-08-24T03:45:02.6030155Z Aug 24 03:45:02 ... 4 more 2022-08-24T03:45:02.6032269Z Aug 24 03:45:02 Caused by: java.lang.RuntimeException: java.util.concurrent.CompletionException: java.lang.RuntimeException: Failed to reload lookup 'FULL' cache. 2022-08-24T03:45:02.6033990Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.LookupFullCache.getIfPresent(LookupFullCache.java:74) 2022-08-24T03:45:02.6035675Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.CachingLookupFunction.lookup(CachingLookupFunction.java:122) 2022-08-24T03:45:02.6037117Z Aug 24 03:45:02 at org.apache.flink.table.functions.LookupFunction.eval(LookupFunction.java:52) 2022-08-24T03:45:02.6038277Z Aug 24 03:45:02 at LookupFunction$109548.flatMap(Unknown Source) 2022-08-24T03:45:02.6039528Z Aug 24 03:45:02 at org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner.doFetch(LookupJoinRunner.java:92) 2022-08-24T03:45:02.6041280Z Aug 24 03:45:02 at org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner.processElement(LookupJoinRunner.java:79) 2022-08-24T03:45:02.6043077Z Aug 24 03:45:02 at org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner.processElement(LookupJoinRunner.java:34) 2022-08-24T03:45:02.6044507Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66) 2022-08-24T03:45:02.6045903Z Aug 24 03:45:02 at org.apache.flink.streaming.runtime.tasks.ChainingOutput.pushToOperator(ChainingOutput.java:99) 2022-08-24T03:45:02.6047336Z Aug 24 03:45:02 at org.apache.flink.streaming.runtime.tasks.ChainingOutput.collect(ChainingOutput.java:80) 2022-08-24T03:45:02.6048783Z Aug 24 03:45:02 at org.apache.flink.streaming.runtime.tasks.ChainingOutput.collect(ChainingOutput.java:39) 2022-08-24T03:45:02.6050162Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56) 2022-08-24T03:45:02.6051690Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29) 2022-08-24T03:45:02.6053386Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418) 2022-08-24T03:45:02.6055157Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513) 2022-08-24T03:45:02.6056876Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103) 2022-08-24T03:45:02.6058505Z Aug 24 03:45:02 at org.apache.flink.streaming.api.functions.source.FromElementsFunction.run(FromElementsFunction.java:231) 2022-08-24T03:45:02.6059974Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110) 2022-08-24T03:45:02.6061347Z Aug 24 03:45:02 at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67) 2022-08-24T03:45:02.6063003Z Aug 24 03:45:02 at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:333) 2022-08-24T03:45:02.6065141Z Aug 24 03:45:02 Caused by: java.util.concurrent.CompletionException: java.lang.RuntimeException: Failed to reload lookup 'FULL' cache. 2022-08-24T03:45:02.6066557Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273) 2022-08-24T03:45:02.6067945Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280) 2022-08-24T03:45:02.6069310Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1643) 2022-08-24T03:45:02.6070702Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$AsyncRun.exec(CompletableFuture.java:1632) 2022-08-24T03:45:02.6072177Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) 2022-08-24T03:45:02.6073457Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) 2022-08-24T03:45:02.6074907Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) 2022-08-24T03:45:02.6076156Z Aug 24 03:45:02 at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) 2022-08-24T03:45:02.6077752Z Aug 24 03:45:02 Caused by: java.lang.RuntimeException: Failed to reload lookup 'FULL' cache. 2022-08-24T03:45:02.6079119Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.CacheLoader.run(CacheLoader.java:105) 2022-08-24T03:45:02.6080604Z Aug 24 03:45:02 at java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640) 2022-08-24T03:45:02.6081691Z Aug 24 03:45:02 ... 5 more 2022-08-24T03:45:02.6083708Z Aug 24 03:45:02 Caused by: java.lang.RuntimeException: Failed to load data into the lookup 'FULL' cache from InputSplit org.apache.flink.table.runtime.functions.table.fullcache.FullCacheTestInputFormat$QueueInputSplit@67854815 2022-08-24T03:45:02.6085871Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.inputformat.InputSplitCacheLoadTask.run(InputSplitCacheLoadTask.java:94) 2022-08-24T03:45:02.6087785Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.inputformat.InputFormatCacheLoader.reloadCache(InputFormatCacheLoader.java:102) 2022-08-24T03:45:02.6089522Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.CacheLoader.run(CacheLoader.java:90) 2022-08-24T03:45:02.6090630Z Aug 24 03:45:02 ... 6 more 2022-08-24T03:45:02.6092180Z Aug 24 03:45:02 Caused by: java.lang.RuntimeException: Could not instantiate generated class 'KeyProjection$109542' 2022-08-24T03:45:02.6093614Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:74) 2022-08-24T03:45:02.6095161Z Aug 24 03:45:02 at org.apache.flink.table.runtime.keyselector.GenericRowDataKeySelector.getKey(GenericRowDataKeySelector.java:52) 2022-08-24T03:45:02.6096823Z Aug 24 03:45:02 at org.apache.flink.table.runtime.keyselector.GenericRowDataKeySelector.getKey(GenericRowDataKeySelector.java:29) 2022-08-24T03:45:02.6098583Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.inputformat.InputSplitCacheLoadTask.run(InputSplitCacheLoadTask.java:79) 2022-08-24T03:45:02.6099844Z Aug 24 03:45:02 ... 8 more 2022-08-24T03:45:02.6101114Z Aug 24 03:45:02 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. 2022-08-24T03:45:02.6102880Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:94) 2022-08-24T03:45:02.6104217Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:101) 2022-08-24T03:45:02.6105525Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:68) 2022-08-24T03:45:02.6106608Z Aug 24 03:45:02 ... 11 more 2022-08-24T03:45:02.6108117Z Aug 24 03:45:02 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. 2022-08-24T03:45:02.6110087Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2051) 2022-08-24T03:45:02.6111720Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962) 2022-08-24T03:45:02.6113370Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859) 2022-08-24T03:45:02.6114888Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92) 2022-08-24T03:45:02.6115931Z Aug 24 03:45:02 ... 13 more 2022-08-24T03:45:02.6117177Z Aug 24 03:45:02 Caused by: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. 2022-08-24T03:45:02.6118673Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:107) 2022-08-24T03:45:02.6120123Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$0(CompileUtils.java:92) 2022-08-24T03:45:02.6121778Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4864) 2022-08-24T03:45:02.6123576Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529) 2022-08-24T03:45:02.6125251Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278) 2022-08-24T03:45:02.6126991Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155) 2022-08-24T03:45:02.6128591Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045) 2022-08-24T03:45:02.6130111Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962) 2022-08-24T03:45:02.6131763Z Aug 24 03:45:02 at org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859) 2022-08-24T03:45:02.6133369Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92) 2022-08-24T03:45:02.6134777Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:101) 2022-08-24T03:45:02.6136236Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:68) 2022-08-24T03:45:02.6137497Z Aug 24 03:45:02 at org.apache.flink.table.runtime.keyselector.GenericRowDataKeySelector.getKey(GenericRowDataKeySelector.java:52) 2022-08-24T03:45:02.6138796Z Aug 24 03:45:02 at org.apache.flink.table.runtime.keyselector.GenericRowDataKeySelector.getKey(GenericRowDataKeySelector.java:29) 2022-08-24T03:45:02.6140234Z Aug 24 03:45:02 at org.apache.flink.table.runtime.functions.table.lookup.fullcache.inputformat.InputSplitCacheLoadTask.run(InputSplitCacheLoadTask.java:79) 2022-08-24T03:45:02.6141457Z Aug 24 03:45:02 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 2022-08-24T03:45:02.6146427Z Aug 24 03:45:02 at java.util.concurrent.FutureTask.run(FutureTask.java:266) 2022-08-24T03:45:02.6147685Z Aug 24 03:45:02 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 2022-08-24T03:45:02.6149049Z Aug 24 03:45:02 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 2022-08-24T03:45:02.6150192Z Aug 24 03:45:02 at java.lang.Thread.run(Thread.java:748) 2022-08-24T03:45:02.6153609Z Aug 24 03:45:02 Caused by: org.codehaus.janino.InternalCompilerException: Compiling "KeyProjection$109542": Trying to access closed classloader. Please check if you store classloaders directly or indirectly in static fields. If the stacktrace suggests that the leak occurs in a third party library and cannot be fixed immediately, you can disable this check with the configuration 'classloader.check-leaked-classloader'. 2022-08-24T03:45:02.6156044Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:382) 2022-08-24T03:45:02.6157265Z Aug 24 03:45:02 at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237) 2022-08-24T03:45:02.6158569Z Aug 24 03:45:02 at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465) 2022-08-24T03:45:02.6159872Z Aug 24 03:45:02 at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216) 2022-08-24T03:45:02.6161097Z Aug 24 03:45:02 at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207) 2022-08-24T03:45:02.6162659Z Aug 24 03:45:02 at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80) 2022-08-24T03:45:02.6163829Z Aug 24 03:45:02 at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75) 2022-08-24T03:45:02.6165126Z Aug 24 03:45:02 at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:104) 2022-08-24T03:45:02.6166156Z Aug 24 03:45:02 ... 19 more 2022-08-24T03:45:02.6168806Z Aug 24 03:45:02 Caused by: java.lang.IllegalStateException: Trying to access closed classloader. Please check if you store classloaders directly or indirectly in static fields. If the stacktrace suggests that the leak occurs in a third party library and cannot be fixed immediately, you can disable this check with the configuration 'classloader.check-leaked-classloader'. 2022-08-24T03:45:02.6171268Z Aug 24 03:45:02 at org.apache.flink.util.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:184) 2022-08-24T03:45:02.6173345Z Aug 24 03:45:02 at org.apache.flink.util.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.loadClass(FlinkUserCodeClassLoaders.java:192) 2022-08-24T03:45:02.6174695Z Aug 24 03:45:02 at java.lang.Class.forName0(Native Method) 2022-08-24T03:45:02.6175663Z Aug 24 03:45:02 at java.lang.Class.forName(Class.java:348) 2022-08-24T03:45:02.6176871Z Aug 24 03:45:02 at org.codehaus.janino.ClassLoaderIClassLoader.findIClass(ClassLoaderIClassLoader.java:89) 2022-08-24T03:45:02.6178209Z Aug 24 03:45:02 at org.codehaus.janino.IClassLoader.loadIClass(IClassLoader.java:312) 2022-08-24T03:45:02.6179482Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.findTypeByName(UnitCompiler.java:8556) 2022-08-24T03:45:02.6180766Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6601) 2022-08-24T03:45:02.6182178Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573) 2022-08-24T03:45:02.6183437Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215) 2022-08-24T03:45:02.6184758Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481) 2022-08-24T03:45:02.6186120Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476) 2022-08-24T03:45:02.6187399Z Aug 24 03:45:02 at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928) 2022-08-24T03:45:02.6188610Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476) 2022-08-24T03:45:02.6189872Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469) 2022-08-24T03:45:02.6191091Z Aug 24 03:45:02 at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927) 2022-08-24T03:45:02.6192407Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) 2022-08-24T03:45:02.6193644Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.access$1300(UnitCompiler.java:215) 2022-08-24T03:45:02.6194938Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$34.getInterfaces2(UnitCompiler.java:10002) 2022-08-24T03:45:02.6196169Z Aug 24 03:45:02 at org.codehaus.janino.IClass.getInterfaces(IClass.java:497) 2022-08-24T03:45:02.6197312Z Aug 24 03:45:02 at org.codehaus.janino.IClass.getIMethods(IClass.java:263) 2022-08-24T03:45:02.6198461Z Aug 24 03:45:02 at org.codehaus.janino.IClass.getIMethods(IClass.java:237) 2022-08-24T03:45:02.6199646Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:492) 2022-08-24T03:45:02.6200857Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432) 2022-08-24T03:45:02.6202211Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215) 2022-08-24T03:45:02.6203556Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411) 2022-08-24T03:45:02.6205035Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406) 2022-08-24T03:45:02.6206461Z Aug 24 03:45:02 at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414) 2022-08-24T03:45:02.6207873Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406) 2022-08-24T03:45:02.6209088Z Aug 24 03:45:02 at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378) 2022-08-24T03:45:02.6210038Z Aug 24 03:45:02 ... 26 more {code} https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=40327&view=logs&j=0c940707-2659-5648-cbe6-a1ad63045f0a&t=075c2716-8010-5565-fe08-3c4bb45824a4&l=21331 -- This message was sent by Atlassian Jira (v8.20.10#820010)