Agreed!
Aljoscha
On 14.10.20 06:38, Jingsong Li wrote:
Hi Aljoscha,
Thanks for your feedback.
Yes, we should add DataStream Providers to the table bridge module.
I think your concerns are right, including the relationship between
DataStream and table.
My understanding is that the parallelism
Hi Aljoscha,
Thanks for your feedback.
Yes, we should add DataStream Providers to the table bridge module.
I think your concerns are right, including the relationship between
DataStream and table.
My understanding is that the parallelism specified by the user is only the
initialization paralleli
Hi Jingsong,
I'm sorry, I didn't want to block you for so long on this. I thought
about it again.
I think it's fine to add a DataStream Provider if this really unblocks
users from migrating to newer Flink versions. I'm guessing you will add
that to the table bridge module?
Regarding the pa
Hi Aljoscha,
I want to separate `Customized parallelism` and `Parallelism inference`.
### Customized parallelism
First, I want to explain the current DataStream parallelism setting:
`env.fromSource(...).setParallelism(...)`.
This is how users explicitly specify parallelism, and it is the only wa
Hi,
I'll only respond regarding the parallelism for now because I need to
think some more about DataStream.
What I'm saying is that exposing a parallelism only for Table Connectors
is not the right thing. If we want to allow sources to tell the
system/framework what would be a good paralleli
Hi everyone,
Thanks for the proposal.
In our company,we meet the same situation as @liu shouwei.
We developed some features base on flink.Such as parallelism of sql source/sink
connector, and kafka delay consumer which is adding a flatmap and a keyby
transformation after the source Datastream.
Hi Jingsong,
Thanks for driving this effort. I have two minor comments.
1. IMHO, parallelism is a concept that applies to all ScanTableSource.
So instead of defining a new interface, is it more natural to incorporate
parallel inference to existing interfaces, e.g. ScanTableSource
or
Hi Aljoscha,
Thank you for your feedback,
## Connector parallelism
Requirements:
Set parallelism by user specified or inferred by connector.
How to configure parallelism in DataStream:
In the DataStream world, the only way to configure parallelism is
`SingleOutputStreamOperator.setParallelism`.
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 a
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)
Hi Jingsong,
Thanks for preparing this FLIP.
WRT ParallelismProvider, it looks good to me.
Kurt Young 于2020年9月24日周四 下午4:14写道:
> Yeah, JDBC is definitely a popular use case we should consider.
>
> Best,
> Kurt
>
>
> On Thu, Sep 24, 2020 at 4:11 PM Flavio Pompermaier
> wrote:
>
> > Hi Kurt, in t
Thanks Kurt and Flavio for your feedback.
To Kurt:
> Briefly introduce use cases and what kind of options are needed in your
opinion.
In the "Choose Scan Parallelism" chapter:
- I explained the user cases
- I adjusted the relationship to make user specified parallelism more
convenient
To Flavio
Yeah, JDBC is definitely a popular use case we should consider.
Best,
Kurt
On Thu, Sep 24, 2020 at 4:11 PM Flavio Pompermaier
wrote:
> Hi Kurt, in the past we had a very interesting use case in this regard: our
> customer (oracle) db was quite big and running too many queries in parallel
> was
Hi Kurt, in the past we had a very interesting use case in this regard: our
customer (oracle) db was quite big and running too many queries in parallel
was too heavy and it was causing the queries to fail.
So we had to limit the source parallelism to 2 threads. After the fetching
of the data the ot
Thanks Jingsong for driving this, this is indeed a useful feature and lots
of users are asking for it.
For setting a fixed source parallelism, I'm wondering whether this is
necessary. For kafka,
I can imagine users would expect Flink will use the number of partitions as
the parallelism. If it's to
+1,it’s a good news
> 2020年9月23日 下午6:22,Jingsong Li 写道:
>
> Hi all,
>
> I'd like to start a discussion about improving the new TableSource and
> TableSink interfaces.
>
> Most connectors have been migrated to FLIP-95, but there are still the
> Filesystem and Hive that have not been migrated. T
Hi all,
I'd like to start a discussion about improving the new TableSource and
TableSink interfaces.
Most connectors have been migrated to FLIP-95, but there are still the
Filesystem and Hive that have not been migrated. They have some
requirements on table connector API. And users also have some
17 matches
Mail list logo