Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
1.14.0
Description
CREATE TABLE MySink ( `a` SMALLINT ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath' ) CREATE TABLE database8_t0 ( `c0` SMALLINT ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath11' ) CREATE TABLE database8_t1 ( `c0` SMALLINT, `c1` TINYINT ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath22' ) INSERT OVERWRITE database8_t0(c0) VALUES(cast(22424 as SMALLINT)) INSERT OVERWRITE database8_t1(c0, c1) VALUES(cast(-17443 as SMALLINT), cast(97 as TINYINT)) insert into MySink SELECT database8_t0.c0 AS ref0 FROM database8_t0, database8_t1 WHERE CAST ((- (database8_t0.c0)) AS BOOLEAN)
After running that , you will get the errors:
2021-06-29 19:39:27 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:440) at sun.reflect.GeneratedMethodAccessor32.invoke(Unknown Source) 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 'BatchExecCalc$4536' at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:66) at org.apache.flink.table.runtime.operators.CodeGenOperatorFactory.createStreamOperator(CodeGenOperatorFactory.java:43) at org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil.createOperator(StreamOperatorFactoryUtil.java:80) at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOperator(OperatorChain.java:626) at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOperatorChain(OperatorChain.java:600) at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:540) at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:171) at org.apache.flink.streaming.runtime.tasks.StreamTask.executeRestore(StreamTask.java:547) at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:646) at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:536) 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:834) 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:78) at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:77) at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:64) ... 12 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:76) ... 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:106) at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:76) 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) ... 17 more Caused by: org.codehaus.commons.compiler.CompileException: Line 53, Column 26: Assignment conversion not possible from type "int" to type "short" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211) at org.codehaus.janino.UnitCompiler.assignmentConversion(UnitCompiler.java:11062) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3790) 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:2476) 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:103) ... 23 more