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

Chengxiang Li commented on FLINK-3226:
--------------------------------------

[~fhueske] and [~twalthr] , this task should be blocked by me for a while, 
sorry about that. The task seems more complicated and has more work than i 
expected, it contains the physical plan genration part of Timo's prototype + 
the whole previous Table API translation part  actually, to move this forward 
faster, as Fabian suggested, i agree that we can split it into multi sub tasks, 
such as expression code generation, and even by the Calcite RelNodes, for 
example, we can work on RelNodes with no dependency in parallel, such as Sort, 
Join and Aggregate, etc. i would make this task very small, which only contains 
the Project translator without expression code generation and make it ready 
during this week.
Besides, while try to test my code, i found the {{PlannerImpl}} check its state 
during each step of query planning process, and there is no way to set it 
manually, for Table API, we actually skip several steps, such as parse, 
validate, it failed while we directly go to the optimize step due to the 
unmatched state, do you guys have any idea about this?

> Translate optimized logical Table API plans into physical plans representing 
> DataSet programs
> ---------------------------------------------------------------------------------------------
>
>                 Key: FLINK-3226
>                 URL: https://issues.apache.org/jira/browse/FLINK-3226
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API
>            Reporter: Fabian Hueske
>            Assignee: Chengxiang Li
>
> This issue is about translating an (optimized) logical Table API (see 
> FLINK-3225) query plan into a physical plan. The physical plan is a 1-to-1 
> representation of the DataSet program that will be executed. This means:
> - Each Flink RelNode refers to exactly one Flink DataSet or DataStream 
> operator.
> - All (join and grouping) keys of Flink operators are correctly specified.
> - The expressions which are to be executed in user-code are identified.
> - All fields are referenced with their physical execution-time index.
> - Flink type information is available.
> - Optional: Add physical execution hints for joins
> The translation should be the final part of Calcite's optimization process.
> For this task we need to:
> - implement a set of Flink DataSet RelNodes. Each RelNode corresponds to one 
> Flink DataSet operator (Map, Reduce, Join, ...). The RelNodes must hold all 
> relevant operator information (keys, user-code expression, strategy hints, 
> parallelism).
> - implement rules to translate optimized Calcite RelNodes into Flink 
> RelNodes. We start with a straight-forward mapping and later add rules that 
> merge several relational operators into a single Flink operator, e.g., merge 
> a join followed by a filter. Timo implemented some rules for the first SQL 
> implementation which can be used as a starting point.
> - Integrate the translation rules into the Calcite optimization process



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to