You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/05/03 13:02:04 UTC

[jira] [Commented] (FLINK-6436) Bug in CommonCorrelate's generateCollector method when using udtf with udf

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

ASF GitHub Bot commented on FLINK-6436:
---------------------------------------

GitHub user godfreyhe opened a pull request:

    https://github.com/apache/flink/pull/3815

    [FLINK-6436] [table] Bug in CommonCorrelate's generateCollector method when using udtf with udf

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/godfreyhe/flink FLINK-6436

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3815.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3815
    
----
commit ae2331d8ff9e9fe61781d1986b3ceff4e060ca96
Author: godfreyhe <go...@163.com>
Date:   2017-05-03T12:59:34Z

    [FLINK-6436] [table] Bug in CommonCorrelate's generateCollector method when using udtf with udf

----


> Bug in CommonCorrelate's generateCollector method when using udtf with udf
> --------------------------------------------------------------------------
>
>                 Key: FLINK-6436
>                 URL: https://issues.apache.org/jira/browse/FLINK-6436
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API & SQL
>            Reporter: godfrey he
>            Assignee: godfrey he
>
> When running table api code (using udtf with udf) like:
> {code}
>     val env = ExecutionEnvironment.getExecutionEnvironment
>     val tableEnv = TableEnvironment.getTableEnvironment(env, config)
>     val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
>     val func0 = new TableFunc0
>     val result = in
>       .join(func0('c))
>       .where(StartWith('name, "J") && ('age > 20))
>       .select('c, 'name, 'age)
>       .toDataSet[Row]
>     val results = result.collect()
> {code}
> An exception will be thrown, as below:
> {code}
> Caused by: java.lang.Exception: The user defined 'open(Configuration)' method in class org.apache.flink.table.runtime.CorrelateFlatMapRunner caused an exception: Table program cannot be compiled. This is a bug. Please file an issue.
> 	at org.apache.flink.runtime.operators.BatchTask.openUserCode(BatchTask.java:1335)
> 	at org.apache.flink.runtime.operators.chaining.ChainedFlatMapDriver.openTask(ChainedFlatMapDriver.java:47)
> 	at org.apache.flink.runtime.operators.BatchTask.openChainedTasks(BatchTask.java:1375)
> 	at org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:129)
> 	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:700)
> 	at java.lang.Thread.run(Thread.java:745)
> 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.codegen.Compiler$class.compile(Compiler.scala:36)
> 	at org.apache.flink.table.runtime.CorrelateFlatMapRunner.compile(CorrelateFlatMapRunner.scala:30)
> 	at org.apache.flink.table.runtime.CorrelateFlatMapRunner.open(CorrelateFlatMapRunner.scala:47)
> 	at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36)
> 	at org.apache.flink.runtime.operators.BatchTask.openUserCode(BatchTask.java:1333)
> 	... 5 more
> Caused by: org.codehaus.commons.compiler.CompileException: Line 72, Column 121: Unknown variable or type "function_org$apache$flink$table$expressions$utils$StartWith$$752fa7801f5195571801442d482dcbc8"
> 	at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:11523)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)