Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 8F309200BA3 for ; Wed, 5 Oct 2016 12:38:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8C4A2160ADB; Wed, 5 Oct 2016 10:38:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 9CE5C160ADE for ; Wed, 5 Oct 2016 12:38:21 +0200 (CEST) Received: (qmail 31483 invoked by uid 500); 5 Oct 2016 10:38:20 -0000 Mailing-List: contact dev-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list dev@flink.apache.org Received: (qmail 31460 invoked by uid 99); 5 Oct 2016 10:38:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Oct 2016 10:38:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 733992C014E for ; Wed, 5 Oct 2016 10:38:20 +0000 (UTC) Date: Wed, 5 Oct 2016 10:38:20 +0000 (UTC) From: "Anton Mushin (JIRA)" To: dev@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (FLINK-4743) The sqrt/power function not accept the real data types. MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 05 Oct 2016 10:38:22 -0000 Anton Mushin created FLINK-4743: ----------------------------------- Summary: The sqrt/power function not accept the real data types. Key: FLINK-4743 URL: https://issues.apache.org/jira/browse/FLINK-4743 Project: Flink Issue Type: Bug Components: Table API & SQL Affects Versions: 1.1.1 Reporter: Anton Mushin At time calculate the sequences sql aggregate functions for real type column, got exception: No applicable constructor/method found for actual parameters "float, java.math.BigDecimal" And for column of integer type the problem does not occur. Code reproduce: {code} @Test def test():Unit={ val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds = env.fromElements( (1.0f, 1), (2.0f, 2)).toTable(tEnv) tEnv.registerTable("MyTable", ds) val sqlQuery = "SELECT " + "SQRT((SUM(a*a) - SUM(a)*SUM(a)/COUNT(a))/COUNT(a)) "+ "from (select _1 as a from MyTable)" tEnv.sql(sqlQuery).toDataSet[Row].collect().foreach(x=>print(s"$x ")) } {code} got exception: {noformat} org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.api.table.runtime.FunctionCompiler$class.compile(FunctionCompiler.scala:37) at org.apache.flink.api.table.runtime.FlatMapRunner.compile(FlatMapRunner.scala:28) at org.apache.flink.api.table.runtime.FlatMapRunner.open(FlatMapRunner.scala:42) at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38) at org.apache.flink.api.common.operators.base.FlatMapOperatorBase.executeOnCollections(FlatMapOperatorBase.java:62) at org.apache.flink.api.common.operators.CollectionExecutor.executeUnaryOperator(CollectionExecutor.java:249) at org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:147) at org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:129) at org.apache.flink.api.common.operators.CollectionExecutor.executeDataSink(CollectionExecutor.java:180) at org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:156) at org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:129) at org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:113) at org.apache.flink.api.java.CollectionEnvironment.execute(CollectionEnvironment.java:35) at org.apache.flink.test.util.CollectionTestEnvironment.execute(CollectionTestEnvironment.java:47) at org.apache.flink.test.util.CollectionTestEnvironment.execute(CollectionTestEnvironment.java:42) at org.apache.flink.api.scala.ExecutionEnvironment.execute(ExecutionEnvironment.scala:637) at org.apache.flink.api.scala.DataSet.collect(DataSet.scala:547) at org.apache.flink.api.scala.batch.sql.AggregationsITCase.test(AggregationsITCase.scala:307) 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:497) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) 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:271) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) at org.junit.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runners.Suite.runChild(Suite.java:127) at org.junit.runners.Suite.runChild(Suite.java:26) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runner.JUnitCore.run(JUnitCore.java:160) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:234) at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:74) 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:497) at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144) Caused by: org.codehaus.commons.compiler.CompileException: Line 118, Column 57: No applicable constructor/method found for actual parameters "float, java.math.BigDecimal"; candidates are: "public static double org.apache.calcite.runtime.SqlFunctions.power(double, double)", "public static double org.apache.calcite.runtime.SqlFunctions.power(long, java.math.BigDecimal)", "public static double org.apache.calcite.runtime.SqlFunctions.power(long, long)" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:10062) at org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:7476) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7346) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7250) at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3862) at org.codehaus.janino.UnitCompiler.access$6900(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$10.visitMethodInvocation(UnitCompiler.java:3263) at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3971) at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:3290) at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:4356) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2669) at org.codehaus.janino.UnitCompiler.access$4500(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$7.visitAssignment(UnitCompiler.java:2619) at org.codehaus.janino.Java$Assignment.accept(Java.java:3402) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2654) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1635) at org.codehaus.janino.UnitCompiler.access$1100(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$4.visitExpressionStatement(UnitCompiler.java:936) at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2094) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:999) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:985) at org.codehaus.janino.UnitCompiler.access$1000(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$4.visitBlock(UnitCompiler.java:935) at org.codehaus.janino.Java$Block.accept(Java.java:2009) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1737) at org.codehaus.janino.UnitCompiler.access$1200(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$4.visitIfStatement(UnitCompiler.java:937) at org.codehaus.janino.Java$IfStatement.accept(Java.java:2154) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:999) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2284) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:822) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:794) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:504) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:390) at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:182) at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:344) at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1136) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:351) at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:319) at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:358) at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:201) at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192) at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84) at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:77) at org.apache.flink.api.table.runtime.FunctionCompiler$class.compile(FunctionCompiler.scala:34) ... 55 more {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)