@Becket: We totally agree that we don't need table specific connectors
during runtime. As Dawid said, the interfaces proposed here are just for
communication with the planner. Once the properties (watermarks,
computed column, filters, projecttion etc.) are negotiated, we can
configure a regular Flink connector.
E.g. setting the watermark assigner and deserialization schema of a
Kafka connector.
For better separation of concerns, Flink connectors should not include
relational interfaces and depend on flink-table. This is the
responsibility of table source/sink.
@Kurt: I would like to mark them @PublicEvolving already because we need
to deprecate the old interfaces as early as possible. We cannot redirect
to @Internal interfaces. They are not marked @Public, so we can still
evolve them. But a core design shift should not happen again, it would
leave a bad impression if we are redesign over and over again. Instead
we should be confident in the current change.
Regards,
Timo
On 24.03.20 09:20, Dawid Wysakowicz wrote:
Hi Becket,
Answering your question, we have the same intention not to duplicate
connectors between datastream and table apis. The interfaces proposed in
the FLIP are a way to describe relational properties of a source. The
intention is as you described to translate all of those expressed as
expressions or other Table specific structures into a DataStream source.
In other words I think what we are doing here is in line with what you
described.
Best,
Dawid
On 24/03/2020 02:23, Becket Qin wrote:
Hi Timo,
Thanks for the proposal. I completely agree that the current Table
connectors could be simplified quite a bit. I haven't finished reading
everything, but here are some quick thoughts.
Actually to me the biggest question is why should there be two different
connector systems for DataStream and Table? What is the fundamental reason
that is preventing us from merging them to one?
The basic functionality of a connector is to provide capabilities to do IO
and Serde. Conceptually, Table connectors should just be DataStream
connectors that are dealing with Rows. It seems that quite a few of the
special connector requirements are just a specific way to do IO / Serde.
Taking SupportsFilterPushDown as an example, imagine we have the following
interface:
interface FilterableSource<PREDICATE> {
void applyFilterable(Supplier<PREDICATE> predicate);
}
And if a ParquetSource would like to support filterable, it will become:
class ParquetSource implements Source, FilterableSource(FilterPredicate> {
...
}
For Table, one just need to provide an predicate supplier that converts an
Expression to the specified predicate type. This has a few benefit:
1. Same unified API for filterable for sources, regardless of DataStream or
Table.
2. The DataStream users now can also use the ExpressionToPredicate
supplier if they want to.
To summarize, my main point is that I am wondering if it is possible to
have a single set of connector interface for both Table and DataStream,
rather than having two hierarchies. I am not 100% sure if this would work,
but if it works, this would be a huge win from both code maintenance and
user experience perspective.
Thanks,
Jiangjie (Becket) Qin
On Tue, Mar 24, 2020 at 2:03 AM Dawid Wysakowicz <dwysakow...@apache.org>
wrote:
Hi Timo,
Thank you for the proposal. I think it is an important improvement that
will benefit many parts of the Table API. The proposal looks really good
to me and personally I would be comfortable with voting on the current
state.
Best,
Dawid
On 23/03/2020 18:53, Timo Walther wrote:
Hi everyone,
I received some questions around how the new interfaces play together
with formats and their factories.
Furthermore, for MySQL or Postgres CDC logs, the format should be able
to return a `ChangelogMode`.
Also, I incorporated the feedback around the factory design in general.
I added a new section `Factory Interfaces` to the design document.
This should be helpful to understand the big picture and connecting
the concepts.
Please let me know what you think?
Thanks,
Timo
On 18.03.20 13:43, Timo Walther wrote:
Hi Benchao,
this is a very good question. I will update the FLIP about this.
The legacy planner will not support the new interfaces. It will only
support the old interfaces. With the next release, I think the Blink
planner is stable enough to be the default one as well.
Regards,
Timo
On 18.03.20 08:45, Benchao Li wrote:
Hi Timo,
Thank you and others for the efforts to prepare this FLIP.
The FLIP LGTM generally.
+1 for moving blink data structures to table-common, it's useful to
udf too
in the future.
A little question is, do we plan to support the new interfaces and data
types in legacy planner?
Or we only plan to support these new interfaces in blink planner.
And using primary keys from DDL instead of derived key information from
each query is also a good idea,
we met some use cases where this does not works very well before.
This FLIP also makes the dependencies of table modules more clear, I
like
it very much.
Timo Walther <twal...@apache.org> 于2020年3月17日周二 上午1:36写道:
Hi everyone,
I'm happy to present the results of long discussions that we had
internally. Jark, Dawid, Aljoscha, Kurt, Jingsong, me, and many more
have contributed to this design document.
We would like to propose new long-term table source and table sink
interfaces:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces
This is a requirement for FLIP-105 and finalizing FLIP-32.
The goals of this FLIP are:
- Simplify the current interface architecture:
- Merge upsert, retract, and append sinks.
- Unify batch and streaming sources.
- Unify batch and streaming sinks.
- Allow sources to produce a changelog:
- UpsertTableSources have been requested a lot by users. Now
is the
time to open the internal planner capabilities via the new interfaces.
- According to FLIP-105, we would like to support changelogs for
processing formats such as Debezium.
- Don't rely on DataStream API for source and sinks:
- According to FLIP-32, the Table API and SQL should be
independent
of the DataStream API which is why the `table-common` module has no
dependencies on `flink-streaming-java`.
- Source and sink implementations should only depend on the
`table-common` module after FLIP-27.
- Until FLIP-27 is ready, we still put most of the interfaces in
`table-common` and strictly separate interfaces that communicate
with a
planner and actual runtime reader/writers.
- Implement efficient sources and sinks without planner dependencies:
- Make Blink's internal data structures available to connectors.
- Introduce stable interfaces for data structures that can be
marked as `@PublicEvolving`.
- Only require dependencies on `flink-table-common` in the
future
It finalizes the concept of dynamic tables and consideres how all
source/sink related classes play together.
We look forward to your feedback.
Regards,
Timo