flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Fabian Hueske (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-10408) Cannot use window information in Table functions
Date Mon, 24 Sep 2018 09:54:00 GMT

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

Fabian Hueske commented on FLINK-10408:
---------------------------------------

The problem might be that the window properties are accessed in a nested expression. {{w}}
is not an actual attribute but rather an identifier for the window group. Therefore, it requires
special validation logic.

As a workaround, could you try to rewrite the query to:
{code:java}
Table table = tableEnv.fromDataSet(source)
        .window(Slide.over("2.minutes").every("1.minute").on("f0").as("w"))
        .groupBy("f1, w")
        .select("f0, f2, w.start as wStart")
        .select("(f2 * (f0 - wStart).exp() / 1.hour).sum" +
                " / ((f0 - wStart).exp() / 1.hour).sum");

{code}

> Cannot use window information in Table functions
> ------------------------------------------------
>
>                 Key: FLINK-10408
>                 URL: https://issues.apache.org/jira/browse/FLINK-10408
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API &amp; SQL
>         Environment: Flink v1.5.3
>            Reporter: Alexis Sarda-Espinosa
>            Priority: Major
>
> Given the examples shown in [here|https://flink.apache.org/news/2017/03/29/table-sql-api-update.html], I
assume the following should work: 
> {code:java}
> ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment();
> BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(execEnv);
> DataSource<Tuple3<Timestamp, String, Double>> source = execEnv.fromElements(
>         new Tuple3<>(Timestamp.valueOf("2018-09-20 22:00:00"), "a", 1.3),
>         new Tuple3<>(Timestamp.valueOf("2018-09-20 22:01:00"), "a", 2.1),
>         new Tuple3<>(Timestamp.valueOf("2018-09-20 22:02:00"), "a", 3.0),
>         new Tuple3<>(Timestamp.valueOf("2018-09-20 22:00:00"), "b", 2.2),
>         new Tuple3<>(Timestamp.valueOf("2018-09-20 22:01:00"), "b", 1.8)
> );
> Table table = tableEnv.fromDataSet(source)
>         .window(Slide.over("2.minutes").every("1.minute").on("f0").as("w"))
>         .groupBy("f1, w")
>         .select("(f2 * (f0 - w.start).exp() / 1.hour).sum" +
>                 " / ((f0 - w.start).exp() / 1.hour).sum");
> tableEnv.toDataSet(table, Row.class).print();
> {code}
> However, I get the following exception:
> {code:java}
> org.apache.flink.table.api.ValidationException: Cannot resolve [w] given input [f0, f1,
f2]
> {code}
> Maybe related to Flink-6618?
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message