You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Aljoscha Krettek <al...@apache.org> on 2020/08/26 15:29:02 UTC

Re: [DISCUSS] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

I added more changes to the FLIP to try and address comments.

You can see the changes from the last version here: 
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158866741&selectedPageVersions=31&selectedPageVersions=27

If no-one objects anymore I would like to proceed to a VOTE soon.

Best,
Aljoscha

On 30.07.20 17:36, Aljoscha Krettek wrote:
> I see, we actually have some thoughts along that line as well. We have 
> ideas about adding such functionality for `Transformation`, which is the 
> graph structure that underlies both the DataStream API and the newer 
> Table API Runner/Planner.
> 
> There a very rough PoC for that available at [1]. It's a very contrived 
> example but it shows off what would be possible. The `Sink` interface 
> here is just a subclass of the general `TransformationApply` [2] and we 
> could envision a `DataStream.apply()` that let's you apply these general 
> transformation "bundles".
> 
> Keep in mind that this is just rough early ideas and the naming/location 
> of things is somewhat rough. And we might not do it like this in the end.
> 
> Best,
> Aljoscha
> 
> [1] 
> https://github.com/aljoscha/flink/blob/poc-transform-apply-sink/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SinkExample.java 
> 
> 
> [2] 
> https://github.com/aljoscha/flink/blob/poc-transform-apply-sink/flink-core/src/main/java/org/apache/flink/api/dag/TransformationApply.java 
> 
> 
> On 30.07.20 17:26, Flavio Pompermaier wrote:
>> We use runCustomOperation to group a set of operators and into a single
>> functional unit, just to make the code more modular..
>> It's very comfortable indeed.
>>
>> On Thu, Jul 30, 2020 at 5:20 PM Aljoscha Krettek <al...@apache.org>
>> wrote:
>>
>>> That is good input! I was not aware that anyone was actually using
>>> `runCustomOperation()`. Out of curiosity, what are you using that for?
>>>
>>> We have definitely thought about the first two points you mentioned,
>>> though. Especially processing-time will make it tricky to define unified
>>> execution semantics.
>>>
>>> Best,
>>> Aljoscha
>>>
>>> On 30.07.20 17:10, Flavio Pompermaier wrote:
>>>> I just wanted to be propositive about missing api.. :D
>>>>
>>>> On Thu, Jul 30, 2020 at 4:29 PM Seth Wiesman <sj...@gmail.com>
>>> wrote:
>>>>
>>>>> +1 Its time to drop DataSet
>>>>>
>>>>> Flavio, those issues are expected. This FLIP isn't just to drop 
>>>>> DataSet
>>>>> but to also add the necessary enhancements to DataStream such that it
>>> works
>>>>> well on bounded input.
>>>>>
>>>>> On Thu, Jul 30, 2020 at 8:49 AM Flavio Pompermaier <
>>> pompermaier@okkam.it>
>>>>> wrote:
>>>>>
>>>>>> Just to contribute to the discussion, when we tried to do the
>>> migration we
>>>>>> faced some problems that could make migration quite difficult.
>>>>>> 1 - It's difficult to test because of
>>>>>> https://issues.apache.org/jira/browse/FLINK-18647
>>>>>> 2 - missing mapPartition
>>>>>> 3 - missing  <X> DataSet<X> runOperation(CustomUnaryOperation<T, X>
>>>>>> operation)
>>>>>>
>>>>>> On Thu, Jul 30, 2020 at 12:40 PM Arvid Heise <ar...@ververica.com>
>>> wrote:
>>>>>>
>>>>>>> +1 of getting rid of the DataSet API. Is DataStream#iterate already
>>>>>>> superseding DataSet iterations or would that also need to be 
>>>>>>> accounted
>>>>>> for?
>>>>>>>
>>>>>>> In general, all surviving APIs should also offer a smooth experience
>>> for
>>>>>>> switching back and forth.
>>>>>>>
>>>>>>> On Thu, Jul 30, 2020 at 9:39 AM Márton Balassi <
>>>>>> balassi.marton@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi All,
>>>>>>>>
>>>>>>>> Thanks for the write up and starting the discussion. I am in 
>>>>>>>> favor of
>>>>>>>> unifying the APIs the way described in the FLIP and deprecating the
>>>>>>> DataSet
>>>>>>>> API. I am looking forward to the detailed discussion of the changes
>>>>>>>> necessary.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Marton
>>>>>>>>
>>>>>>>> On Wed, Jul 29, 2020 at 12:46 PM Aljoscha Krettek <
>>>>>> aljoscha@apache.org>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Everyone,
>>>>>>>>>
>>>>>>>>> my colleagues (in cc) and I would like to propose this FLIP for
>>>>>>>>> discussion. In short, we want to reduce the number of APIs that we
>>>>>> have
>>>>>>>>> by deprecating the DataSet API. This is a big step for Flink, 
>>>>>>>>> that's
>>>>>> why
>>>>>>>>> I'm also cross-posting this to the User Mailing List.
>>>>>>>>>
>>>>>>>>> FLIP-131: http://s.apache.org/FLIP-131
>>>>>>>>>
>>>>>>>>> I'm posting the introduction of the FLIP below but please refer to
>>>>>> the
>>>>>>>>> document linked above for the full details:
>>>>>>>>>
>>>>>>>>> -- 
>>>>>>>>> Flink provides three main SDKs/APIs for writing Dataflow Programs:
>>>>>> Table
>>>>>>>>> API/SQL, the DataStream API, and the DataSet API. We believe that
>>>>>> this
>>>>>>>>> is one API too many and propose to deprecate the DataSet API in
>>>>>> favor of
>>>>>>>>> the Table API/SQL and the DataStream API. Of course, this is 
>>>>>>>>> easier
>>>>>> said
>>>>>>>>> than done, so in the following, we will outline why we think that
>>>>>> having
>>>>>>>>> too many APIs is detrimental to the project and community. We will
>>>>>> then
>>>>>>>>> describe how we can enhance the Table API/SQL and the 
>>>>>>>>> DataStream API
>>>>>> to
>>>>>>>>> subsume the DataSet API's functionality.
>>>>>>>>>
>>>>>>>>> In this FLIP, we will not describe all the technical details of 
>>>>>>>>> how
>>>>>> the
>>>>>>>>> Table API/SQL and DataStream will be enhanced. The goal is to
>>> achieve
>>>>>>>>> consensus on the idea of deprecating the DataSet API. There will
>>>>>> have to
>>>>>>>>> be follow-up FLIPs that describe the necessary changes for the 
>>>>>>>>> APIs
>>>>>> that
>>>>>>>>> we maintain.
>>>>>>>>> -- 
>>>>>>>>>
>>>>>>>>> Please let us know if you have any concerns or comments. Also,
>>> please
>>>>>>>>> keep discussion to this ML thread instead of commenting in the 
>>>>>>>>> Wiki
>>>>>> so
>>>>>>>>> that we can have a consistent view of the discussion.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Aljoscha
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>> -- 
>>>>>>>
>>>>>>> Arvid Heise | Senior Java Developer
>>>>>>>
>>>>>>> <https://www.ververica.com/>
>>>>>>>
>>>>>>> Follow us @VervericaData
>>>>>>>
>>>>>>> -- 
>>>>>>>
>>>>>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>>>>>>> Conference
>>>>>>>
>>>>>>> Stream Processing | Event Driven | Real Time
>>>>>>>
>>>>>>> -- 
>>>>>>>
>>>>>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>>>>>>
>>>>>>> -- 
>>>>>>> Ververica GmbH
>>>>>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>>>>>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason,
>>> Ji
>>>>>>> (Toni) Cheng
>>>>>
>>>>>
>>>>
>>>
>>>
>>
>