Hi Jark,
Glad to see your feedback!
That's Correct, The proposal is aiming to extend the functionality for
Table API! I like add "drop" to fit the use case you mentioned. Not only
that, if a 100-columns Table. and our UDF needs these 100 columns, we don't
want to define the eval as eval(column0...column99), we prefer to define
eval as eval(Row)。Using it like this: table.select(udf (*)). All we also
need to consider if we put the columns package as a row. In a scenario like
this, we have Classification it as cloumn operation, and  list the changes
to the column operation after the map/flatMap/agg/flatAgg phase is
completed. And Currently,  Xiaowei has started a threading outlining which
talk about what we are proposing. Please see the detail in the mail thread:
Please see the detail in the mail thread:
 
https://mail.google.com/mail/u/0/#search/xiaowei/FMfcgxvzLWzfvCnmvMzzSfxHTSfdwLkB
<https://mail.google.com/mail/u/0/#search/xiaowei/FMfcgxvzLWzfvCnmvMzzSfxHTSfdwLkB>
 .

At this stage the Table API Enhancement Outline as follows:
https://docs.google.com/document/d/1tnpxg31EQz2-MEzSotwFzqatsB4rNLz0I-l_vPa5H4Q/edit?usp=sharing

Please let we know if you have further thoughts or feedback!

Thanks,
Jincheng


Jark Wu <imj...@gmail.com> 于2018年11月6日周二 下午3:35写道:

> Hi jingcheng,
>
> Thanks for your proposal. I think it is a helpful enhancement for TableAPI
> which is a solid step forward for TableAPI.
> It doesn't weaken SQL or DataStream, because the conversion between
> DataStream and Table still works.
> People with advanced cases (e.g. complex and fine-grained state control)
> can go with DataStream,
> but most general cases can stay in TableAPI. This works is aiming to extend
> the functionality for TableAPI,
> to extend the usage scenario, to help TableAPI becomes a more widely used
> API.
>
> For example, someone want to drop one column from a 100-columns Table.
> Currently, we have to convert
> Table to DataStream and use MapFunction to do that, or select the remaining
> 99 columns using Table.select API.
> But if we support Table.drop() method for TableAPI, it will be a very
> convenient method and let users stay in Table.
>
> Looking forward to the more detailed design and further discussion.
>
> Regards,
> Jark
>
> jincheng sun <sunjincheng...@gmail.com> 于2018年11月6日周二 下午1:05写道:
>
> > Hi Rong Rong,
> >
> > Sorry for the late reply, And thanks for your feedback!  We will continue
> > to add more convenience features to the TableAPI, such as map, flatmap,
> > agg, flatagg, iteration etc. And I am very happy that you are interested
> on
> > this proposal. Due to this is a long-term continuous work, we will push
> it
> > in stages.  Currently  Xiaowei has started a threading outlining which
> talk
> > about what we are proposing. Please see the detail in the mail thread:
> >
> >
> https://mail.google.com/mail/u/0/#search/xiaowei/FMfcgxvzLWzfvCnmvMzzSfxHTSfdwLkB
> > <
> >
> https://mail.google.com/mail/u/0/#search/xiaowei/FMfcgxvzLWzfvCnmvMzzSfxHTSfdwLkB
> > >
> >  .
> >
> > The Table API Enhancement Outline as follows:
> >
> >
> https://docs.google.com/document/d/1tnpxg31EQz2-MEzSotwFzqatsB4rNLz0I-l_vPa5H4Q/edit?usp=sharing
> >
> > Please let we know if you have further thoughts or feedback!
> >
> > Thanks,
> > Jincheng
> >
> > Fabian Hueske <fhue...@gmail.com> 于2018年11月5日周一 下午7:03写道:
> >
> > > Hi Jincheng,
> > >
> > > Thanks for this interesting proposal.
> > > I like that we can push this effort forward in a very fine-grained
> > manner,
> > > i.e., incrementally adding more APIs to the Table API.
> > >
> > > However, I also have a few questions / concerns.
> > > Today, the Table API is tightly integrated with the DataSet and
> > DataStream
> > > APIs. It is very easy to convert a Table into a DataSet or DataStream
> and
> > > vice versa. This mean it is already easy to combine custom logic an
> > > relational operations. What I like is that several aspects are clearly
> > > separated like retraction and timestamp handling (see below) + all
> > > libraries on DataStream/DataSet can be easily combined with relational
> > > operations.
> > > I can see that adding more functionality to the Table API would remove
> > the
> > > distinction between DataSet and DataStream. However, wouldn't we get a
> > > similar benefit by extending the DataStream API for proper support for
> > > bounded streams (as is the long-term goal of Flink)?
> > > I'm also a bit skeptical about the optimization opportunities we would
> > > gain. Map/FlatMap UDFs are black boxes that cannot be easily removed
> > > without additional information (I did some research on this a few years
> > ago
> > > [1]).
> > >
> > > Moreover, I think there are a few tricky details that need to be
> resolved
> > > to enable a good integration.
> > >
> > > 1) How to deal with retraction messages? The DataStream API does not
> > have a
> > > notion of retractions. How would a MapFunction or FlatMapFunction
> handle
> > > retraction? Do they need to be aware of the change flag? Custom
> windowing
> > > and aggregation logic would certainly need to have that information.
> > > 2) How to deal with timestamps? The DataStream API does not give access
> > to
> > > timestamps. In the Table API / SQL these are exposed as regular
> > attributes.
> > > How can we ensure that timestamp attributes remain valid (i.e. aligned
> > with
> > > watermarks) if the output is produced by arbitrary code?
> > > There might be more issues of this kind.
> > >
> > > My main question would be how much would we gain with this proposal
> over
> > a
> > > tight integration of Table API and DataStream API, assuming that batch
> > > functionality is moved to DataStream?
> > >
> > > Best, Fabian
> > >
> > > [1] http://stratosphere.eu/assets/papers/openingTheBlackBoxes_12.pdf
> > >
> > >
> > > Am Mo., 5. Nov. 2018 um 02:49 Uhr schrieb Rong Rong <
> walter...@gmail.com
> > >:
> > >
> > > > Hi Jincheng,
> > > >
> > > > Thank you for the proposal! I think being able to define a process /
> > > > co-process function in table API definitely opens up a whole new
> level
> > of
> > > > applications using a unified API.
> > > >
> > > > In addition, as Tzu-Li and Hequn have mentioned, the benefit of
> > > > optimization layer of Table API will already bring in additional
> > benefit
> > > > over directly programming on top of DataStream/DataSet API. I am very
> > > > interested an looking forward to seeing the support for more complex
> > use
> > > > cases, especially iterations. It will enable table API to define much
> > > > broader, event-driven use cases such as real-time ML
> > prediction/training.
> > > >
> > > > As Timo mentioned, This will make Table API diverge from the SQL API.
> > But
> > > > as from my experience Table API was always giving me the impression
> to
> > > be a
> > > > more sophisticated, syntactic-aware way to express relational
> > operations.
> > > > Looking forward to further discussion and collaborations on the FLIP
> > doc.
> > > >
> > > > --
> > > > Rong
> > > >
> > > > On Sun, Nov 4, 2018 at 5:22 PM jincheng sun <
> sunjincheng...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi tison,
> > > > >
> > > > > Thanks a lot for your feedback!
> > > > > I am very happy to see that community contributors agree to
> enhanced
> > > the
> > > > > TableAPI. This work is a long-term continuous work, we will push it
> > in
> > > > > stages, we will soon complete  the enhanced list of the first
> phase,
> > we
> > > > can
> > > > > go deep discussion  in google doc. thanks again for joining on the
> > very
> > > > > important discussion of the Flink Table API.
> > > > >
> > > > > Thanks,
> > > > > Jincheng
> > > > >
> > > > > Tzu-Li Chen <wander4...@gmail.com> 于2018年11月2日周五 下午1:49写道:
> > > > >
> > > > > > Hi jingchengm
> > > > > >
> > > > > > Thanks a lot for your proposal! I find it is a good start point
> for
> > > > > > internal optimization works and help Flink to be more
> > > > > > user-friendly.
> > > > > >
> > > > > > AFAIK, DataStream is the most popular API currently that Flink
> > > > > > users should describe their logic with detailed logic.
> > > > > > From a more internal view the conversion from DataStream to
> > > > > > JobGraph is quite mechanically and hard to be optimized. So when
> > > > > > users program with DataStream, they have to learn more internals
> > > > > > and spend a lot of time to tune for performance.
> > > > > > With your proposal, we provide enhanced functionality of Table
> API,
> > > > > > so that users can describe their job easily on Table aspect. This
> > > gives
> > > > > > an opportunity to Flink developers to introduce an optimize phase
> > > > > > while transforming user program(described by Table API) to
> internal
> > > > > > representation.
> > > > > >
> > > > > > Given a user who want to start using Flink with simple ETL,
> > > pipelining
> > > > > > or analytics, he would find it is most naturally described by
> > > SQL/Table
> > > > > > API. Further, as mentioned by @hequn,
> > > > > >
> > > > > > SQL is a widely used language. It follows standards, is a
> > > > > > > descriptive language, and is easy to use
> > > > > >
> > > > > >
> > > > > > thus we could expect with the enhancement of SQL/Table API, Flink
> > > > > > becomes more friendly to users.
> > > > > >
> > > > > > Looking forward to the design doc/FLIP!
> > > > > >
> > > > > > Best,
> > > > > > tison.
> > > > > >
> > > > > >
> > > > > > jincheng sun <sunjincheng...@gmail.com> 于2018年11月2日周五 上午11:46写道:
> > > > > >
> > > > > > > Hi Hequn,
> > > > > > > Thanks for your feedback! And also thanks for our offline
> > > discussion!
> > > > > > > You are right, unification of batch and streaming is very
> > important
> > > > for
> > > > > > > flink API.
> > > > > > > We will provide more detailed design later, Please let me know
> if
> > > you
> > > > > > have
> > > > > > > further thoughts or feedback.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jincheng
> > > > > > >
> > > > > > > Hequn Cheng <chenghe...@gmail.com> 于2018年11月2日周五 上午10:02写道:
> > > > > > >
> > > > > > > > Hi Jincheng,
> > > > > > > >
> > > > > > > > Thanks a lot for your proposal. It is very encouraging!
> > > > > > > >
> > > > > > > > As we all know, SQL is a widely used language. It follows
> > > > standards,
> > > > > > is a
> > > > > > > > descriptive language, and is easy to use. A powerful feature
> of
> > > SQL
> > > > > is
> > > > > > > that
> > > > > > > > it supports optimization. Users only need to care about the
> > logic
> > > > of
> > > > > > the
> > > > > > > > program. The underlying optimizer will help users optimize
> the
> > > > > > > performance
> > > > > > > > of the program. However, in terms of functionality and ease
> of
> > > use,
> > > > > in
> > > > > > > some
> > > > > > > > scenarios sql will be limited, as described in Jincheng's
> > > proposal.
> > > > > > > >
> > > > > > > > Correspondingly, the DataStream/DataSet api can provide
> > powerful
> > > > > > > > functionalities. Users can write
> > > ProcessFunction/CoProcessFunction
> > > > > and
> > > > > > > get
> > > > > > > > the timer. Compared with SQL, it provides more
> functionalities
> > > and
> > > > > > > > flexibilities. However, it does not support optimization like
> > > SQL.
> > > > > > > > Meanwhile, DataStream/DataSet api has not been unified which
> > > means,
> > > > > for
> > > > > > > the
> > > > > > > > same logic, users need to write a job for each stream and
> > batch.
> > > > > > > >
> > > > > > > > With TableApi, I think we can combine the advantages of both.
> > > Users
> > > > > can
> > > > > > > > easily write relational operations and enjoy optimization. At
> > the
> > > > > same
> > > > > > > > time, it supports more functionality and ease of use. Looking
> > > > forward
> > > > > > to
> > > > > > > > the detailed design/FLIP.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Hequn
> > > > > > > >
> > > > > > > > On Fri, Nov 2, 2018 at 9:48 AM Shaoxuan Wang <
> > > wshaox...@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Aljoscha,
> > > > > > > > > Glad that you like the proposal. We have completed the
> > > prototype
> > > > of
> > > > > > > most
> > > > > > > > > new proposed functionalities. Once collect the feedback
> from
> > > > > > community,
> > > > > > > > we
> > > > > > > > > will come up with a concrete FLIP/design doc.
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Shaoxuan
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Nov 1, 2018 at 8:12 PM Aljoscha Krettek <
> > > > > aljos...@apache.org
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jincheng,
> > > > > > > > > >
> > > > > > > > > > these points sound very good! Are there any concrete
> > > proposals
> > > > > for
> > > > > > > > > > changes? For example a FLIP/design document?
> > > > > > > > > >
> > > > > > > > > > See here for FLIPs:
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Aljoscha
> > > > > > > > > >
> > > > > > > > > > > On 1. Nov 2018, at 12:51, jincheng sun <
> > > > > sunjincheng...@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > *--------I am sorry for the formatting of the email
> > > content.
> > > > I
> > > > > > > > reformat
> > > > > > > > > > > the **content** as follows-----------*
> > > > > > > > > > >
> > > > > > > > > > > *Hi ALL,*
> > > > > > > > > > >
> > > > > > > > > > > With the continuous efforts from the community, the
> Flink
> > > > > system
> > > > > > > has
> > > > > > > > > been
> > > > > > > > > > > continuously improved, which has attracted more and
> more
> > > > users.
> > > > > > > Flink
> > > > > > > > > SQL
> > > > > > > > > > > is a canonical, widely used relational query language.
> > > > However,
> > > > > > > there
> > > > > > > > > are
> > > > > > > > > > > still some scenarios where Flink SQL failed to meet
> user
> > > > needs
> > > > > in
> > > > > > > > terms
> > > > > > > > > > of
> > > > > > > > > > > functionality and ease of use, such as:
> > > > > > > > > > >
> > > > > > > > > > > *1. In terms of functionality*
> > > > > > > > > > >    Iteration, user-defined window, user-defined join,
> > > > > > user-defined
> > > > > > > > > > > GroupReduce, etc. Users cannot express them with SQL;
> > > > > > > > > > >
> > > > > > > > > > > *2. In terms of ease of use*
> > > > > > > > > > >
> > > > > > > > > > >   - Map - e.g. “dataStream.map(mapFun)”. Although
> > > > > > > > “table.select(udf1(),
> > > > > > > > > > >   udf2(), udf3()....)” can be used to accomplish the
> same
> > > > > > > function.,
> > > > > > > > > > with a
> > > > > > > > > > >   map() function returning 100 columns, one has to
> define
> > > or
> > > > > call
> > > > > > > 100
> > > > > > > > > > UDFs
> > > > > > > > > > >   when using SQL, which is quite involved.
> > > > > > > > > > >   - FlatMap -  e.g. “dataStrem.flatmap(flatMapFun)”.
> > > > Similarly,
> > > > > > it
> > > > > > > > can
> > > > > > > > > be
> > > > > > > > > > >   implemented with “table.join(udtf).select()”.
> However,
> > it
> > > > is
> > > > > > > > obvious
> > > > > > > > > > that
> > > > > > > > > > >   dataStream is easier to use than SQL.
> > > > > > > > > > >
> > > > > > > > > > > Due to the above two reasons, some users have to use
> the
> > > > > > DataStream
> > > > > > > > API
> > > > > > > > > > or
> > > > > > > > > > > the DataSet API. But when they do that, they lose the
> > > > > unification
> > > > > > > of
> > > > > > > > > > batch
> > > > > > > > > > > and streaming. They will also lose the sophisticated
> > > > > > optimizations
> > > > > > > > such
> > > > > > > > > > as
> > > > > > > > > > > codegen, aggregate join transpose and multi-stage agg
> > from
> > > > > Flink
> > > > > > > SQL.
> > > > > > > > > > >
> > > > > > > > > > > We believe that enhancing the functionality and
> > > productivity
> > > > is
> > > > > > > vital
> > > > > > > > > for
> > > > > > > > > > > the successful adoption of Table API. To this end,
> Table
> > > API
> > > > > > still
> > > > > > > > > > > requires more efforts from every contributor in the
> > > > community.
> > > > > We
> > > > > > > see
> > > > > > > > > > great
> > > > > > > > > > > opportunity in improving our user’s experience from
> this
> > > > work.
> > > > > > Any
> > > > > > > > > > feedback
> > > > > > > > > > > is welcome.
> > > > > > > > > > >
> > > > > > > > > > > Regards,
> > > > > > > > > > >
> > > > > > > > > > > Jincheng
> > > > > > > > > > >
> > > > > > > > > > > jincheng sun <sunjincheng...@gmail.com> 于2018年11月1日周四
> > > > > 下午5:07写道:
> > > > > > > > > > >
> > > > > > > > > > >> Hi all,
> > > > > > > > > > >>
> > > > > > > > > > >> With the continuous efforts from the community, the
> > Flink
> > > > > system
> > > > > > > has
> > > > > > > > > > been
> > > > > > > > > > >> continuously improved, which has attracted more and
> more
> > > > > users.
> > > > > > > > Flink
> > > > > > > > > > SQL
> > > > > > > > > > >> is a canonical, widely used relational query language.
> > > > > However,
> > > > > > > > there
> > > > > > > > > > are
> > > > > > > > > > >> still some scenarios where Flink SQL failed to meet
> user
> > > > needs
> > > > > > in
> > > > > > > > > terms
> > > > > > > > > > of
> > > > > > > > > > >> functionality and ease of use, such as:
> > > > > > > > > > >>
> > > > > > > > > > >>
> > > > > > > > > > >>   -
> > > > > > > > > > >>
> > > > > > > > > > >>   In terms of functionality
> > > > > > > > > > >>
> > > > > > > > > > >> Iteration, user-defined window, user-defined join,
> > > > > user-defined
> > > > > > > > > > >> GroupReduce, etc. Users cannot express them with SQL;
> > > > > > > > > > >>
> > > > > > > > > > >>   -
> > > > > > > > > > >>
> > > > > > > > > > >>   In terms of ease of use
> > > > > > > > > > >>   -
> > > > > > > > > > >>
> > > > > > > > > > >>      Map - e.g. “dataStream.map(mapFun)”. Although
> > > > > > > > > “table.select(udf1(),
> > > > > > > > > > >>      udf2(), udf3()....)” can be used to accomplish
> the
> > > same
> > > > > > > > > function.,
> > > > > > > > > > with a
> > > > > > > > > > >>      map() function returning 100 columns, one has to
> > > define
> > > > > or
> > > > > > > call
> > > > > > > > > > 100 UDFs
> > > > > > > > > > >>      when using SQL, which is quite involved.
> > > > > > > > > > >>      -
> > > > > > > > > > >>
> > > > > > > > > > >>      FlatMap -  e.g. “dataStrem.flatmap(flatMapFun)”.
> > > > > Similarly,
> > > > > > > it
> > > > > > > > > can
> > > > > > > > > > >>      be implemented with “table.join(udtf).select()”.
> > > > However,
> > > > > > it
> > > > > > > is
> > > > > > > > > > obvious
> > > > > > > > > > >>      that datastream is easier to use than SQL.
> > > > > > > > > > >>
> > > > > > > > > > >>
> > > > > > > > > > >> Due to the above two reasons, some users have to use
> the
> > > > > > > DataStream
> > > > > > > > > API
> > > > > > > > > > or
> > > > > > > > > > >> the DataSet API. But when they do that, they lose the
> > > > > > unification
> > > > > > > of
> > > > > > > > > > batch
> > > > > > > > > > >> and streaming. They will also lose the sophisticated
> > > > > > optimizations
> > > > > > > > > such
> > > > > > > > > > as
> > > > > > > > > > >> codegen, aggregate join transpose  and multi-stage agg
> > > from
> > > > > > Flink
> > > > > > > > SQL.
> > > > > > > > > > >>
> > > > > > > > > > >> We believe that enhancing the functionality and
> > > productivity
> > > > > is
> > > > > > > > vital
> > > > > > > > > > for
> > > > > > > > > > >> the successful adoption of Table API. To this end,
> > Table
> > > > API
> > > > > > > still
> > > > > > > > > > >> requires more efforts from every contributor in the
> > > > community.
> > > > > > We
> > > > > > > > see
> > > > > > > > > > great
> > > > > > > > > > >> opportunity in improving our user’s experience from
> this
> > > > work.
> > > > > > Any
> > > > > > > > > > feedback
> > > > > > > > > > >> is welcome.
> > > > > > > > > > >>
> > > > > > > > > > >> Regards,
> > > > > > > > > > >>
> > > > > > > > > > >> Jincheng
> > > > > > > > > > >>
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to