Timo, Thanks for the discussion I have only read the "Conversion of DataStream to Table" part so i would only put some objections there ~
> StreamTableEnvironment.fromInsertStream(DataStream<T>): Table At first glance, from the perspective of a user, i'm confused by why we must dintinguish on the API level what a data stream is, e.g. an insert stream or whatever other kind of stream. As a user, he does not expect to must distinguish between several datastream options. The framework should have the ability to infer the ChangelogMode of the stream, but sadly we can not at the moment, becase we do not have a metadata to describe the ChangelogMode what actually the framework need. And could it be: StreamTableEnvironment.fromDataStream(DataStream<T>, ChangelogMode) where the ChanglogMode is optional because 90% of the datastream are insert for now. or: DataStream.withChangelogMode(ChangelogMode) so that DataStream can be self-describing what kind of stream it is (again, if not specified, the default is INSERT). > tEnv >.fromInsertStream(DataStream<T>) >.select('*, system_rowtime().as("rowtime"), system_proctime().as(“proctime”)) In order to declare the time-attributes on datastream, i must say I prefer tEnv.fromDataStream(dataStream, Schema) for these reasons: - Schema is the uniform interface to declare the metadata for a table in the Table/SQL API, with an imperative coding style, in Descriptor API we also use it for the time-attributes purpose - Use a projection for time-attributes is not a good idea, because from the SQL side, we declare it as a metadata of part of the table schema when we define the DDL. Although we may explain the DDL internally using computed column, that does not mean we must do that in the DataStream API explicitly. In the SQL world, no projection function outputs type of time-attribute, we better still put the time-attributes in the scope of the table metadata. Best, Danny Chan 在 2020年8月19日 +0800 PM4:22,Timo Walther <twal...@apache.org>,写道: > Hi everyone, > > I would like to propose a FLIP that aims to resolve the remaining > shortcomings in the Table API: > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-136%3A++Improve+interoperability+between+DataStream+and+Table+API > > The Table API has received many new features over the last year. It > supports a new type system (FLIP-37), connectors support changelogs > (FLIP-95), we have well defined internal data structures (FLIP-95), > support for result retrieval in an interactive fashion (FLIP-84), and > soon new TableDescriptors (FLIP-129). > > However, the interfaces from and to DataStream API have not been touched > during the introduction of these new features and are kind of outdated. > The interfaces lack important functionality that is available in Table > API but not exposed to DataStream API users. DataStream API is still our > most important API which is why a good interoperability is crucial. > > This FLIP is a mixture of different topics that improve the > interoperability between DataStream and Table API in terms of: > > - DataStream <-> Table conversion > - translation of type systems TypeInformation <-> DataType > - schema definition (incl. rowtime, watermarks, primary key) > - changelog handling > - row handling in DataStream API > > I'm looking forward to your feedback. > > Regards, > Timo