Hi David,

thanks for your feedback. Feedback from someone who interacts with many users is very valuable. I added an explanation for StatementSets to the FLIP.

Regarding watermarks and fromInsertStream, actually the

`Schema.watermark("ts", system_watermark())`

is not really necessary in the `fromChangelogStream`. It is added to satify the Schema interface and be similar to SQL DDL.

We could already extract the watermark strategy if we see `system_rowtime()` because in most of the cases we will simply use the DataStream API watermarks.

But maybe some users want to generate watermarks after preprocessing in DataStream API. In this cases users what to define a computed watermark expression.

So for simplicity in the Simple API we introduce:

tEnv
  .fromInsertStream(DataStream<T>)
.select('*, system_rowtime().as("rowtime"), system_proctime().as("proctime"))

and just rely on the watermarks that travel through DataStream API already. I added another comment to the FLIP.

Regards,
Timo


On 19.08.20 10:53, David Anderson wrote:
Timo, nice to see this.

As someone who expects to use these interfaces, but who doesn't fully
understand the existing Table API, I like what I see. Just a couple of
comments:

The way that watermarks fit into the fromChangelogStream case makes sense
to me, and I'm wondering why watermarks don't come up in the previous
section about fromInsertStream.

I wasn't familiar with StatementSets, and I couldn't find an explanation in
the docs. I eventually found this short paragraph in an email from Fabian
Hueske, which clarified everything in that section for me:

     FLIP-84 [1] added the concept of a "statement set" to group multiple
INSERT
     INTO statements (SQL or Table API) together. The statements in a
statement
     set are jointly optimized and executed as a single Flink job.

Maybe if you add this to the FLIP it will help other readers as well.

Best,
David

On Wed, Aug 19, 2020 at 10:22 AM Timo Walther <twal...@apache.org> wrote:

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



Reply via email to