Sorry Jingsong but I didn't understand your reply..Can you better explain
the following sentences please? Probably I miss some Table API background
here (I used only JDBOutputFormat).
"We can not use a simple "scan.query.statement", because in
JDBCTableSource, it also deal with project pushdown too. Which means that
the select part can not be modified casual.
Maybe you can configure a rich table name for this."

I can take care of opening tickets but I need to understand exactly how
many and I need to be sure of explaining the problem with the correct terms.

Best,
Flavio

On Wed, Apr 22, 2020 at 11:52 AM Jingsong Li <jingsongl...@gmail.com> wrote:

> Thanks for the explanation.
> You can create JIRA for this.
>
> For "SELECT public.A.x, public.B.y FROM public.A JOIN public.B on
> public.A.pk <http://public.a.pk/> = public.B.fk <http://public.b.fk/>. "
> We can not use a simple "scan.query.statement", because in
> JDBCTableSource, it also deal with project pushdown too. Which means that
> the select part can not be modified casual.
> Maybe you can configure a rich table name for this.
>
> Best,
> Jingsong Lee
>
> On Wed, Apr 22, 2020 at 5:24 PM Flavio Pompermaier <pomperma...@okkam.it>
> wrote:
>
>> Because in my use case the parallelism was not based on a range on
>> keys/numbers but on a range of dates, so I needed a custom Parameter
>> Provider.
>> For what regards pushdown I don't know how Flink/Blink currently
>> works..for example, let's say I have a Postgres catalog containing 2 tables
>> (public.A and public.B).
>> If I do the following query : SELECT public.A.x, public.B.y FROM public.A
>> JOIN public.B on public.A.pk = public.B.fk.
>> Will this be pushdown as a single query or will Flink fetch both tables
>> and the perform the join?
>> Talking with Bowen I understood that to avoid this I could define a VIEW
>> in the db (but this is not alway possible) or in Flink (but from what I
>> know this is very costly).
>> In this case a parameter "scan.query.statement" without a
>> "scan.parameter.values.provider.class" is super helpful and could improve
>> performance a lot!
>>
>> On Wed, Apr 22, 2020 at 11:06 AM Jingsong Li <jingsongl...@gmail.com>
>> wrote:
>>
>>> Hi,
>>>
>>> You are right about the lower and upper, it is a must to parallelize the
>>> fetch of the data.
>>> And filter pushdown is used to filter more data at JDBC server.
>>>
>>> Yes, we can provide "scan.query.statement" and
>>> "scan.parameter.values.provider.class" for jdbc connector. But maybe we
>>> need be careful about this too flexible API.
>>>
>>> Can you provide more about your case? Why can not been solved by lower
>>> and upper with filter pushdown?
>>>
>>> Best,
>>> Jingsong Lee
>>>
>>> On Wed, Apr 22, 2020 at 4:45 PM Flavio Pompermaier <pomperma...@okkam.it>
>>> wrote:
>>>
>>>> Maybe I am wrong but support pushdown for JDBC is one thing (that is
>>>> probably useful) while parameters providers are required if you want to
>>>> parallelize the fetch of the data.
>>>> You are not mandated to use NumericBetweenParametersProvider, you can
>>>> use the ParametersProvider you prefer, depending on the statement you have.
>>>> Or do you have in mind something else?
>>>>
>>>> On Wed, Apr 22, 2020 at 10:33 AM Jingsong Li <jingsongl...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> Now in JDBCTableSource.getInputFormat, It's written explicitly: WHERE
>>>>> XXX BETWEEN ? AND ?. So we must use `NumericBetweenParametersProvider`.
>>>>> I don't think this is a good and long-term solution.
>>>>> I think we should support filter push-down for JDBCTableSource, so in
>>>>> this way, we can write the filters that we want, what do you think?
>>>>>
>>>>> Best,
>>>>> Jingsong Lee
>>>>>
>>>>>
>>>>> On Tue, Apr 21, 2020 at 10:00 PM Flavio Pompermaier <
>>>>> pomperma...@okkam.it> wrote:
>>>>>
>>>>>> Hi all,
>>>>>> we have a use case where we have a prepared statement that we
>>>>>> parameterize using a custom parameters provider (similar to what happens 
>>>>>> in
>>>>>> testJDBCInputFormatWithParallelismAndNumericColumnSplitting[1]).
>>>>>> How can we handle this using the JDBC table API?
>>>>>> What should we do to handle such a use case? Is there anyone willing
>>>>>> to mentor us in its implementation?
>>>>>>
>>>>>> Another question: why flink-jdbc has not been renamed to
>>>>>> flink-connector-jdbc?
>>>>>>
>>>>>> Thanks in advance,
>>>>>> Flavio
>>>>>>
>>>>>> [1]
>>>>>> https://github.com/apache/flink/blob/master/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Best, Jingsong Lee
>>>>>
>>>>
>>>>
>>>
>>> --
>>> Best, Jingsong Lee
>>>
>>
>
> --
> Best, Jingsong Lee
>

Reply via email to