Hi Benoît,

Thanks for the reminder. I will look into the issue and hopefully we can
target it into 1.9.2 and 1.10.

Cheers,
Jark

On Fri, 3 Jan 2020 at 18:21, Benoît Paris <
benoit.pa...@centraliens-lille.org> wrote:

> >  If anyone finds that blink planner has any significant defects and has
> a larger regression than the old planner, please let us know.
>
> Overall, the Blink-exclusive features are must (TopN, deduplicate,
> LAST_VALUE, plan reuse, etc)! But all use cases of the Legacy planner in
> production are not covered:
> An edge case of Temporal Table Functions does not allow computed Tables
> (as opposed to TableSources) to be used on the query side in Blink (
> https://issues.apache.org/jira/browse/FLINK-14200)
>
> Cheers
> Ben
>
>
> On Fri, Jan 3, 2020 at 10:00 AM Jeff Zhang <zjf...@gmail.com> wrote:
>
>> +1, I have already made blink as the default planner of flink interpreter
>> in Zeppelin
>>
>>
>> Jingsong Li <jingsongl...@gmail.com> 于2020年1月3日周五 下午4:37写道:
>>
>>> Hi Jark,
>>>
>>> +1 for default blink planner in SQL-CLI.
>>> I believe this new planner can be put into practice in production.
>>> We've worked hard for nearly a year, but the old planner didn't move on.
>>>
>>> And I'd like to cc to user@flink.apache.org.
>>> If anyone finds that blink planner has any significant defects and has a
>>> larger regression than the old planner, please let us know. We will be very
>>> grateful.
>>>
>>> Best,
>>> Jingsong Lee
>>>
>>> On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu <xbjt...@gmail.com> wrote:
>>>
>>>> +1 for this.
>>>> We bring many SQL/API features and enhance stability in 1.10 release,
>>>> and almost all of them happens in Blink planner.
>>>> SQL CLI is the most convenient entrypoint for me, I believe many users
>>>> will have a better experience If we set Blink planner as default planner.
>>>>
>>>> Best,
>>>> Leonard
>>>>
>>>> > 在 2020年1月3日,15:16,Terry Wang <zjuwa...@gmail.com> 写道:
>>>> >
>>>> > Since what blink planner can do is a superset of flink planner, big
>>>> +1 for changing the default planner to Blink planner from my side.
>>>> >
>>>> > Best,
>>>> > Terry Wang
>>>> >
>>>> >
>>>> >
>>>> >> 2020年1月3日 15:00,Jark Wu <imj...@gmail.com> 写道:
>>>> >>
>>>> >> Hi everyone,
>>>> >>
>>>> >> In 1.10 release, Flink SQL supports many awesome features and
>>>> improvements,
>>>> >> including:
>>>> >> - support watermark statement and computed column in DDL
>>>> >> - fully support all data types in Hive
>>>> >> - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
>>>> >> - support INSERT OVERWRITE and INSERT PARTITION
>>>> >>
>>>> >> However, all the features and improvements are only avaiable in Blink
>>>> >> planner, not in Old planner.
>>>> >> There are also some other features are limited in Blink planner, e.g.
>>>> >> Dimension Table Join [1],
>>>> >> TopN [2], Deduplicate [3], streaming aggregates optimization [4],
>>>> and so on.
>>>> >>
>>>> >> But Old planner is still the default planner in Table API & SQL. It
>>>> is
>>>> >> frustrating for users to set
>>>> >> to blink planner manually when every time start a SQL CLI. And it's
>>>> >> surprising to see unsupported
>>>> >> exception if they trying out the new features but not switch planner.
>>>> >>
>>>> >> SQL CLI is a very important entrypoint for trying out new feautures
>>>> and
>>>> >> prototyping for users.
>>>> >> In order to give new planner more exposures, I would like to suggest
>>>> to set
>>>> >> default planner
>>>> >> for SQL Client to Blink planner before 1.10 release.
>>>> >>
>>>> >> The approach is just changing the default SQL CLI yaml
>>>> configuration[5]. In
>>>> >> this way, the existing
>>>> >> environment is still compatible and unaffected.
>>>> >>
>>>> >> Changing the default planner for the whole Table API & SQL is
>>>> another topic
>>>> >> and is out of scope of this discussion.
>>>> >>
>>>> >> What do you think?
>>>> >>
>>>> >> Best,
>>>> >> Jark
>>>> >>
>>>> >> [1]:
>>>> >>
>>>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
>>>> >> [2]:
>>>> >>
>>>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
>>>> >> [3]:
>>>> >>
>>>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
>>>> >> [4]:
>>>> >>
>>>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
>>>> >> [5]:
>>>> >>
>>>> https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
>>>> >
>>>>
>>>>
>>>
>>> --
>>> Best, Jingsong Lee
>>>
>>
>>
>> --
>> Best Regards
>>
>> Jeff Zhang
>>
>
>
> --
> Benoît Paris
> Ingénieur Machine Learning Explicable
> Tél : +33 6 60 74 23 00
> http://benoit.paris
> http://explicable.ml
>

Reply via email to