Caizhi Weng created FLINK-29142:
-----------------------------------

             Summary: round(float, int) fails to compile in Flink 1.14
                 Key: FLINK-29142
                 URL: https://issues.apache.org/jira/browse/FLINK-29142
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Runtime
    Affects Versions: 1.14.5
            Reporter: Caizhi Weng


Add the following test to {{TableEnvironmentITCase}} to reproduce this bug.

{code:scala}
@Test
def myTest(): Unit = {
  tEnv.executeSql(
    """
      |create table T (
      |  a float
      |) with (
      |  'connector' = 'datagen',
      |  'number-of-rows' = '3'
      |)
      |""".stripMargin)
  tEnv.executeSql("select round(a, 2) from T").print()
}
{code}

The exception is
{code}
/* 1 */
/* 2 */      public class StreamExecCalc$3 extends 
org.apache.flink.table.runtime.operators.TableStreamOperator
/* 3 */          implements 
org.apache.flink.streaming.api.operators.OneInputStreamOperator {
/* 4 */
/* 5 */        private final Object[] references;
/* 6 */        org.apache.flink.table.data.BoxedWrapperRowData out = new 
org.apache.flink.table.data.BoxedWrapperRowData(1);
/* 7 */        private final 
org.apache.flink.streaming.runtime.streamrecord.StreamRecord outElement = new 
org.apache.flink.streaming.runtime.streamrecord.StreamRecord(null);
/* 8 */
/* 9 */        public StreamExecCalc$3(
/* 10 */            Object[] references,
/* 11 */            org.apache.flink.streaming.runtime.tasks.StreamTask task,
/* 12 */            org.apache.flink.streaming.api.graph.StreamConfig config,
/* 13 */            org.apache.flink.streaming.api.operators.Output output,
/* 14 */            
org.apache.flink.streaming.runtime.tasks.ProcessingTimeService 
processingTimeService) throws Exception {
/* 15 */          this.references = references;
/* 16 */          
/* 17 */          this.setup(task, config, output);
/* 18 */          if (this instanceof 
org.apache.flink.streaming.api.operators.AbstractStreamOperator) {
/* 19 */            
((org.apache.flink.streaming.api.operators.AbstractStreamOperator) this)
/* 20 */              .setProcessingTimeService(processingTimeService);
/* 21 */          }
/* 22 */        }
/* 23 */
/* 24 */        @Override
/* 25 */        public void open() throws Exception {
/* 26 */          super.open();
/* 27 */          
/* 28 */        }
/* 29 */
/* 30 */        @Override
/* 31 */        public void 
processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord 
element) throws Exception {
/* 32 */          org.apache.flink.table.data.RowData in1 = 
(org.apache.flink.table.data.RowData) element.getValue();
/* 33 */          
/* 34 */          float field$0;
/* 35 */          boolean isNull$0;
/* 36 */          boolean isNull$1;
/* 37 */          float result$2;
/* 38 */          
/* 39 */          
/* 40 */          isNull$0 = in1.isNullAt(0);
/* 41 */          field$0 = -1.0f;
/* 42 */          if (!isNull$0) {
/* 43 */            field$0 = in1.getFloat(0);
/* 44 */          }
/* 45 */          
/* 46 */          out.setRowKind(in1.getRowKind());
/* 47 */          
/* 48 */          
/* 49 */          
/* 50 */          
/* 51 */          
/* 52 */          
/* 53 */          isNull$1 = isNull$0 || false;
/* 54 */          result$2 = -1.0f;
/* 55 */          if (!isNull$1) {
/* 56 */            
/* 57 */            result$2 = 
/* 58 */          org.apache.calcite.runtime.SqlFunctions.sround(field$0, 
((int) 2))
/* 59 */                     ;
/* 60 */            
/* 61 */          }
/* 62 */          
/* 63 */          if (isNull$1) {
/* 64 */            out.setNullAt(0);
/* 65 */          } else {
/* 66 */            out.setFloat(0, result$2);
/* 67 */          }
/* 68 */                    
/* 69 */                  
/* 70 */          output.collect(outElement.replace(out));
/* 71 */          
/* 72 */          
/* 73 */        }
/* 74 */
/* 75 */        
/* 76 */
/* 77 */        @Override
/* 78 */        public void close() throws Exception {
/* 79 */           super.close();
/* 80 */          
/* 81 */        }
/* 82 */
/* 83 */        
/* 84 */      }
/* 85 */    


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 
org.apache.flink.table.api.TableEnvironmentITCase.myTest(TableEnvironmentITCase.scala:98)
        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:59)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        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:258)
        at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
        at 
org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
        at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
        at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
        at 
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
        at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
        at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
        at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
        at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
        at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
        at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
        at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
        at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
        at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
        at org.junit.runners.Suite.runChild(Suite.java:128)
        at org.junit.runners.Suite.runChild(Suite.java:27)
        at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
        at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
        at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
        at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
        at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
        at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
        at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
        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)
        ... 43 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)
        ... 45 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)
        ... 45 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:252)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:242)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:233)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:684)
        at 
org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:444)
        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.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)
        at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
        at 
java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
        at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
        at 
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
Caused by: java.lang.RuntimeException: Could not instantiate generated class 
'StreamExecCalc$3'
        at 
org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:85)
        at 
org.apache.flink.table.runtime.operators.CodeGenOperatorFactory.createStreamOperator(CodeGenOperatorFactory.java:40)
        at 
org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil.createOperator(StreamOperatorFactoryUtil.java:81)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.createOperator(OperatorChain.java:712)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.createOperatorChain(OperatorChain.java:686)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:626)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:187)
        at 
org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.<init>(RegularOperatorChain.java:63)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:663)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:651)
        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.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:102)
        at 
org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:83)
        ... 14 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)
        ... 16 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)
        ... 19 more
Caused by: org.codehaus.commons.compiler.CompileException: Line 57, Column 23: 
Assignment conversion not possible from type "double" to type "float"
        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:86)
        ... 25 more
{code}



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

Reply via email to