You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Runkang He (Jira)" <ji...@apache.org> on 2022/12/21 10:20:00 UTC

[jira] [Commented] (FLINK-25802) OverWindow in batch mode failed

    [ https://issues.apache.org/jira/browse/FLINK-25802?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17650804#comment-17650804 ] 

Runkang He commented on FLINK-25802:
------------------------------------

Hi, [~TsReaper], would you like to assign this to me, I'm glad to fix this bug.

> OverWindow in batch mode failed
> -------------------------------
>
>                 Key: FLINK-25802
>                 URL: https://issues.apache.org/jira/browse/FLINK-25802
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner, Table SQL / Runtime
>    Affects Versions: 1.14.0
>            Reporter: Zoyo Pei
>            Priority: Major
>
> {code:java}
> StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
> env.setRuntimeMode(RuntimeExecutionMode.BATCH);
> StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
> DataStream<Row> userStream = env
>         .fromElements(
>                 Row.of(LocalDateTime.parse("2021-08-21T13:00:00"), 1, "Alice"),
>                 Row.of(LocalDateTime.parse("2021-08-21T13:05:00"), 2, "Bob"),
>                 Row.of(LocalDateTime.parse("2021-08-21T13:10:00"), 2, "Bob"))
>         .returns(
>                 Types.ROW_NAMED(
>                         new String[]{"ts", "uid", "name"},
>                         Types.LOCAL_DATE_TIME, Types.INT, Types.STRING));
> tEnv.createTemporaryView(
>         "UserTable",
>         userStream,
>         Schema.newBuilder()
>                 .column("ts", DataTypes.TIMESTAMP(3))
>                 .column("uid", DataTypes.INT())
>                 .column("name", DataTypes.STRING())
>                 .watermark("ts", "ts - INTERVAL '1' SECOND")
>                 .build());
> String statement = "SELECT name, ts, COUNT(name) OVER w AS cnt FROM UserTable " +
>         "WINDOW w AS (" +
>         " PARTITION BY name" +
>         " ORDER BY ts" +
>         " RANGE BETWEEN INTERVAL '10' MINUTE PRECEDING AND CURRENT ROW" +
>         ")";
> tEnv.executeSql(statement).print();
>  {code}
>  
> {code:java}
> /* 1 */
> /* 2 */      public class RangeBoundComparator$38 implements org.apache.flink.table.runtime.generated.RecordComparator {
> /* 3 */
> /* 4 */        private final Object[] references;
> /* 5 */        
> /* 6 */
> /* 7 */        public RangeBoundComparator$38(Object[] references) {
> /* 8 */          this.references = references;
> /* 9 */          
> /* 10 */          
> /* 11 */        }
> /* 12 */
> /* 13 */        @Override
> /* 14 */        public int compare(org.apache.flink.table.data.RowData in1, org.apache.flink.table.data.RowData in2) {
> /* 15 */          
> /* 16 */                  org.apache.flink.table.data.TimestampData field$39;
> /* 17 */                  boolean isNull$39;
> /* 18 */                  org.apache.flink.table.data.TimestampData field$40;
> /* 19 */                  boolean isNull$40;
> /* 20 */                  isNull$39 = in1.isNullAt(0);
> /* 21 */                  field$39 = null;
> /* 22 */                  if (!isNull$39) {
> /* 23 */                    field$39 = in1.getTimestamp(0, 3);
> /* 24 */                  }
> /* 25 */                  isNull$40 = in2.isNullAt(0);
> /* 26 */                  field$40 = null;
> /* 27 */                  if (!isNull$40) {
> /* 28 */                    field$40 = in2.getTimestamp(0, 3);
> /* 29 */                  }
> /* 30 */                  if (isNull$39 && isNull$40) {
> /* 31 */                     return 1;
> /* 32 */                  } else if (isNull$39 || isNull$40) {
> /* 33 */                     return -1;
> /* 34 */                  } else {
> /* 35 */                     
> /* 36 */                            
> /* 37 */                            long result$41;
> /* 38 */                            boolean isNull$41;
> /* 39 */                            long result$42;
> /* 40 */                            boolean isNull$42;
> /* 41 */                            boolean isNull$43;
> /* 42 */                            long result$44;
> /* 43 */                            boolean isNull$45;
> /* 44 */                            boolean result$46;
> /* 45 */                            isNull$41 = (java.lang.Long) field$39 == null;
> /* 46 */                            result$41 = -1L;
> /* 47 */                            if (!isNull$41) {
> /* 48 */                              result$41 = (java.lang.Long) field$39;
> /* 49 */                            }
> /* 50 */                            isNull$42 = (java.lang.Long) field$40 == null;
> /* 51 */                            result$42 = -1L;
> /* 52 */                            if (!isNull$42) {
> /* 53 */                              result$42 = (java.lang.Long) field$40;
> /* 54 */                            }
> /* 55 */                            
> /* 56 */                            
> /* 57 */                            
> /* 58 */                            
> /* 59 */                            isNull$43 = isNull$41 || isNull$42;
> /* 60 */                            result$44 = -1L;
> /* 61 */                            if (!isNull$43) {
> /* 62 */                              
> /* 63 */                              result$44 = (long) (result$41 - result$42);
> /* 64 */                              
> /* 65 */                            }
> /* 66 */                            
> /* 67 */                            
> /* 68 */                            isNull$45 = isNull$43 || false;
> /* 69 */                            result$46 = false;
> /* 70 */                            if (!isNull$45) {
> /* 71 */                              
> /* 72 */                              result$46 = result$44 >= ((int) -600000);
> /* 73 */                              
> /* 74 */                            }
> /* 75 */                            
> /* 76 */                            if (result$46) {
> /* 77 */                              return 1;
> /* 78 */                            } else {
> /* 79 */                              return -1;
> /* 80 */                            }
> /* 81 */                          
> /* 82 */                  }
> /* 83 */               
> /* 84 */        }
> /* 85 */      }
> /* 86 */      /* 1 */
> /* 2 */      public class RangeBoundComparator$38 implements org.apache.flink.table.runtime.generated.RecordComparator {
> /* 3 */
> /* 4 */        private final Object[] references;
> /* 5 */        
> /* 6 */
> /* 7 */        public RangeBoundComparator$38(Object[] references) {
> /* 8 */          this.references = references;
> /* 9 */          
> /* 10 */          
> /* 11 */        }
> /* 12 */
> /* 13 */        @Override
> /* 14 */        public int compare(org.apache.flink.table.data.RowData in1, org.apache.flink.table.data.RowData in2) {
> /* 15 */          
> /* 16 */                  org.apache.flink.table.data.TimestampData field$39;
> /* 17 */                  boolean isNull$39;
> /* 18 */                  org.apache.flink.table.data.TimestampData field$40;
> /* 19 */                  boolean isNull$40;
> /* 20 */                  isNull$39 = in1.isNullAt(0);
> /* 21 */                  field$39 = null;
> /* 22 */                  if (!isNull$39) {
> /* 23 */                    field$39 = in1.getTimestamp(0, 3);
> /* 24 */                  }
> /* 25 */                  isNull$40 = in2.isNullAt(0);
> /* 26 */                  field$40 = null;
> /* 27 */                  if (!isNull$40) {
> /* 28 */                    field$40 = in2.getTimestamp(0, 3);
> /* 29 */                  }
> /* 30 */                  if (isNull$39 && isNull$40) {
> /* 31 */                     return 1;
> /* 32 */                  } else if (isNull$39 || isNull$40) {
> /* 33 */                     return -1;
> /* 34 */                  } else {
> /* 35 */                     
> /* 36 */                            
> /* 37 */                            long result$41;
> /* 38 */                            boolean isNull$41;
> /* 39 */                            long result$42;
> /* 40 */                            boolean isNull$42;
> /* 41 */                            boolean isNull$43;
> /* 42 */                            long result$44;
> /* 43 */                            boolean isNull$45;
> /* 44 */                            boolean result$46;
> /* 45 */                            isNull$41 = (java.lang.Long) field$39 == null;
> /* 46 */                            result$41 = -1L;
> /* 47 */                            if (!isNull$41) {
> /* 48 */                              result$41 = (java.lang.Long) field$39;
> /* 49 */                            }
> /* 50 */                            isNull$42 = (java.lang.Long) field$40 == null;
> /* 51 */                            result$42 = -1L;
> /* 52 */                            if (!isNull$42) {
> /* 53 */                              result$42 = (java.lang.Long) field$40;
> /* 54 */                            }
> /* 55 */                            
> /* 56 */                            
> /* 57 */                            
> /* 58 */                            
> /* 59 */                            isNull$43 = isNull$41 || isNull$42;
> /* 60 */                            result$44 = -1L;
> /* 61 */                            if (!isNull$43) {
> /* 62 */                              
> /* 63 */                              result$44 = (long) (result$41 - result$42);
> /* 64 */                              
> /* 65 */                            }
> /* 66 */                            
> /* 67 */                            
> /* 68 */                            isNull$45 = isNull$43 || false;
> /* 69 */                            result$46 = false;
> /* 70 */                            if (!isNull$45) {
> /* 71 */                              
> /* 72 */                              result$46 = result$44 >= ((int) -600000);
> /* 73 */                              
> /* 74 */                            }
> /* 75 */                            
> /* 76 */                            if (result$46) {
> /* 77 */                              return 1;
> /* 78 */                            } else {
> /* 79 */                              return -1;
> /* 80 */                            }
> /* 81 */                          
> /* 82 */                  }
> /* 83 */               
> /* 84 */        }
> /* 85 */      }
> /* 86 */      Exception in thread "main" 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.utils.PrintUtils.printAsTableauForm(PrintUtils.java:152)
>     at org.apache.flink.table.api.internal.TableResultImpl.print(TableResultImpl.java:160)
>     at OverWindowIssue01.main(OverWindowIssue01.java:44)
> 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)
>     ... 5 more
> Caused by: java.util.concurrent.ExecutionException: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
>     at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395)
>     at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2022)
>     at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:175)
>     ... 7 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.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
>     at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
>     at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
>     at org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:250)
>     at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
>     at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
>     at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
>     at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
>     at org.apache.flink.util.concurrent.FutureUtils.doForward(FutureUtils.java:1389)
>     at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$null$1(ClassLoadingUtils.java:93)
>     at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68)
>     at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$guardCompletionWithContextClassLoader$2(ClassLoadingUtils.java:92)
>     at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
>     at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
>     at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
>     at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
>     at org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$1.onComplete(AkkaFutureUtils.java:47)
>     at akka.dispatch.OnComplete.internal(Future.scala:300)
>     at akka.dispatch.OnComplete.internal(Future.scala:297)
>     at akka.dispatch.japi$CallbackBridge.apply(Future.scala:224)
>     at akka.dispatch.japi$CallbackBridge.apply(Future.scala:221)
>     at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60)
>     at org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$DirectExecutionContext.execute(AkkaFutureUtils.java:65)
>     at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68)
>     at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284)
>     at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284)
>     at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284)
>     at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:621)
>     at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:24)
>     at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:23)
>     at scala.concurrent.Future.$anonfun$andThen$1(Future.scala:532)
>     at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:29)
>     at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:29)
>     at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60)
>     at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:63)
>     at akka.dispatch.BatchingExecutor$BlockableBatch.$anonfun$run$1(BatchingExecutor.scala:100)
>     at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
>     at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:81)
>     at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:100)
>     at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:49)
>     at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:48)
>     at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)
>     at java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)
>     at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)
>     at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)
>     at java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)
> 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:228)
>     at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:218)
>     at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:209)
>     at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:679)
>     at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
>     at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:444)
>     at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.base/java.lang.reflect.Method.invoke(Method.java:566)
>     at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:316)
>     at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
>     at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:314)
>     at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217)
>     at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78)
>     at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163)
>     at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)
>     at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)
>     at scala.PartialFunction.applyOrElse(PartialFunction.scala:123)
>     at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)
>     at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)
>     at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>     at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
>     at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
>     at akka.actor.Actor.aroundReceive(Actor.scala:537)
>     at akka.actor.Actor.aroundReceive$(Actor.scala:535)
>     at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)
>     at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580)
>     at akka.actor.ActorCell.invoke(ActorCell.scala:548)
>     at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)
>     at akka.dispatch.Mailbox.run(Mailbox.scala:231)
>     at akka.dispatch.Mailbox.exec(Mailbox.scala:243)
>     ... 5 more
> Caused by: java.lang.RuntimeException: Could not instantiate generated class 'RangeBoundComparator$38'
>     at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:75)
>     at org.apache.flink.table.runtime.operators.over.frame.RangeSlidingOverFrame.open(RangeSlidingOverFrame.java:63)
>     at org.apache.flink.table.runtime.operators.over.BufferDataOverWindowOperator.open(BufferDataOverWindowOperator.java:92)
>     at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:110)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:711)
>     at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:687)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:654)
>     at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958)
>     at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927)
>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575)
>     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:76)
>     at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:102)
>     at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:69)
>     ... 12 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:74)
>     ... 14 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.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)
>     ... 17 more
> Caused by: org.codehaus.commons.compiler.CompileException: Line 45, Column 66: Cannot cast "org.apache.flink.table.data.TimestampData" to "java.lang.Long"
>     at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211)
>     at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5051)
>     at org.codehaus.janino.UnitCompiler.access$8600(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4418)
>     at org.codehaus.janino.UnitCompiler$16.visitCast(UnitCompiler.java:4396)
>     at org.codehaus.janino.Java$Cast.accept(Java.java:4898)
>     at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4396)
>     at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5662)
>     at org.codehaus.janino.UnitCompiler.compileBoolean2(UnitCompiler.java:4120)
>     at org.codehaus.janino.UnitCompiler.access$6600(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$14.visitBinaryOperation(UnitCompiler.java:3957)
>     at org.codehaus.janino.UnitCompiler$14.visitBinaryOperation(UnitCompiler.java:3935)
>     at org.codehaus.janino.Java$BinaryOperation.accept(Java.java:4864)
>     at org.codehaus.janino.UnitCompiler.compileBoolean(UnitCompiler.java:3935)
>     at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:4448)
>     at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5004)
>     at org.codehaus.janino.UnitCompiler.access$8500(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$16.visitBinaryOperation(UnitCompiler.java:4417)
>     at org.codehaus.janino.UnitCompiler$16.visitBinaryOperation(UnitCompiler.java:4396)
>     at org.codehaus.janino.Java$BinaryOperation.accept(Java.java:4864)
>     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.compile2(UnitCompiler.java:1553)
>     at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1493)
>     at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1487)
>     at org.codehaus.janino.Java$Block.accept(Java.java:2779)
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487)
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2468)
>     at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1495)
>     at org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1487)
>     at org.codehaus.janino.Java$IfStatement.accept(Java.java:2950)
>     at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487)
>     at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2468)
>     at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:215)
>     at org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1495)
>     at org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1487)
>     at org.codehaus.janino.Java$IfStatement.accept(Java.java:2950)
>     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)
>     ... 23 more {code}
>  
> But when I run in streaming mode, it was successful.



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