Hiļ¼ŒAljoscha, I would like to share a use case to second setting parallelism
of table sink(or limiting parallelism range of table sink): When writing
data to databases, there is limitation for number of jdbc connections and
query TPS. we would get errors of too many connections or high load for
db and poor performance because of too many small requests if the optimizer
didn't know such information, and set a large parallelism for sink when
matching the parallelism of its input.

On Thu, 24 Sep 2020 at 22:41, Aljoscha Krettek <aljos...@apache.org> wrote:

> Thanks for the proposal! I think the use cases that we are trying to
> solve are indeed valid. However, I think we might have to take a step
> back to look at what we're trying to solve and how we can solve it.
>
> The FLIP seems to have two broader topics: 1) add "get parallelism" to
> sinks/sources 2) let users write DataStream topologies for
> sinks/sources. I'll treat them separately below.
>
> I think we should not add "get parallelism" to the Table Sink API
> because I think it's the wrong level of abstraction. The Table API
> connectors are (or should be) more or less thin wrappers around
> "physical" connectors. By "physical" I mean the underlying (mostly
> DataStream API) connectors. For example, with the Kafka Connector the
> Table API connector just does the configuration parsing and determines a
> good (de)serialization format and then creates the underlying
> FlinkKafkaConsumer/FlinkKafkaProducer.
>
> If we wanted to add a "get parallelism" it would be in those underlying
> connectors but I'm also skeptical about adding such a method there
> because it is a static assignment and would preclude clever
> optimizations about the parallelism of a connector at runtime. But maybe
> that's thinking too much about future work so I'm open to discussion there.
>
> Regarding the second point of letting Table connector developers use
> DataStream: I think we should not do it. One of the purposes of FLIP-95
> [1] was to decouple the Table API from the DataStream API for the basic
> interfaces. Coupling the two too closely at that basic level will make
> our live harder in the future when we want to evolve those APIs or when
> we want the system to be better at choosing how to execute sources and
> sinks. An example of this is actually the past of the Table API. Before
> FLIP-95 we had connectors that dealt directly with DataSet and
> DataStream, meaning that if users wanted their Table Sink to work in
> both BATCH and STREAMING mode they had to provide two implementations.
> The trend is towards unifying the sources/sinks to common interfaces
> that can be used for both BATCH and STREAMING execution but, again, I
> think exposing DataStream here would be a step back in the wrong direction.
>
> I think the solution to the existing user requirement of using
> DataStream sources and sinks with the Table API should be better
> interoperability between the two APIs, which is being tackled right now
> in FLIP-136 [2]. If FLIP-136 is not adequate for the use cases that
> we're trying to solve here, maybe we should think about FLIP-136 some more.
>
> What do you think?
>
> Best,
> Aljoscha
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces
> [2]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-136%3A++Improve+interoperability+between+DataStream+and+Table+API
>
>

Reply via email to