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 too
large, one can use the max parallelism to make it smaller.
But for ES, which doesn't have ability to decide a reasonable parallelism
on its own, it might make sense
to introduce a user specified parallelism for such table source.

So I think it would be better to reorganize the document a little bit, to
explain the connectors one by one. Briefly
introduce use cases and what kind of options are needed in your opinion.

Regarding the interface `DataStreamScanProvider`, a concrete example would
help the discussion. What kind
of scenarios do you want to support? And what kind of connectors need such
an interface?

Best,
Kurt


On Wed, Sep 23, 2020 at 9:30 PM admin <17626017...@163.com> wrote:

> +1,it’s a good news
>
> > 2020年9月23日 下午6:22,Jingsong Li <jingsongl...@gmail.com> 写道:
> >
> > 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 additional
> > requirements:
> > - Some connectors have the ability to infer parallelism, the parallelism
> is
> > good for most cases.
> > - Users have customized parallelism configuration requirements for source
> > and sink.
> > - The connectors need to use topology to build their source/sink instead
> of
> > a single function. Like JIRA[1], Partition Commit feature and File
> > Compaction feature.
> >
> > Details are in [2].
> >
> > [1]https://issues.apache.org/jira/browse/FLINK-18674
> > [2]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-146%3A+Improve+new+TableSource+and+TableSink+interfaces
> >
> > Best,
> > Jingsong
>
>

Reply via email to