[
https://issues.apache.org/jira/browse/FLINK-33365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17780456#comment-17780456
]
macdoor615 edited comment on FLINK-33365 at 10/27/23 4:36 PM:
--------------------------------------------------------------
[~davidradl] I create another case
1. create a paimon table and insert one line of data
{code:java}
create table svc1_paimon_prod.dxx.a( ip string, proctime as proctime() );
insert into svc1_paimon_prod.dxx.a values ('10.10.10.10'); {code}
2. create a mysql table and insert two lines of data
{code:java}
create table gem_tmp.b (ip varchar(20), type int);
insert into gem_tmp.b values ('10.10.10.10', 0);
insert into gem_tmp.b values ('10.10.10.10', 1); {code}
3. execute sql in sql-client.sh with flink-connector-jdbc-3.0.0-1.16.jar and.
flink-connector-jdbc-3.1.1-1.17.jar
{code:java}
SELECT * FROM svc1_paimon_prod.dxx.a left join bnpmp_mysql_test.gem_tmp.b FOR
SYSTEM_TIME AS OF a.proctime on b.type = 0 and a.ip = b.ip; {code}
4. screenshot flink-connector-jdbc-3.0.0-1.16.jar, I get one line output ,this
is correct
5. screenshot flink-connector-jdbc-3.1.1-1.17.jar, I get two lines output, this
is incorrect with wrong execution plan
was (Author: macdoor615):
[~davidradl] I create another case
1. create a paimon table and insert one line of data
{code:java}
create table svc1_paimon_prod.dxx.a( ip string, proctime as proctime() );
insert into svc1_paimon_prod.dxx.a values ('10.10.10.10'); {code}
2. create a mysql table and insert two lines of data
{code:java}
create table gem_tmp.b (ip varchar(20), type int);
insert into gem_tmp.b values ('10.10.10.10', 0);
insert into gem_tmp.b values ('10.10.10.10', 1); {code}
3. execute sql in sql-client.sh with flink-connector-jdbc-3.0.0-1.16.jar and.
flink-connector-jdbc-3.1.1-1.17.jar
{code:java}
SELECT * FROM svc1_paimon_prod.dxx.a left join bnpmp_mysql_test.gem_tmp.b FOR
SYSTEM_TIME AS OF a.proctime on b.type = 0 and a.ip = b.ip; {code}
4. screenshot flink-connector-jdbc-3.0.0-1.16.jar, I get one line output ,this
is correct
5. screenshot flink-connector-jdbc-3.1.1-1.17.jar, I get two lines output, this
is incorrect with wrong execution plan
> Missing filter condition in execution plan containing lookup join with mysql
> jdbc connector
> -------------------------------------------------------------------------------------------
>
> Key: FLINK-33365
> URL: https://issues.apache.org/jira/browse/FLINK-33365
> Project: Flink
> Issue Type: Bug
> Components: Connectors / JDBC
> Affects Versions: 1.18.0, 1.17.1
> Environment: Flink 1.17.1 & Flink 1.18.0 with
> flink-connector-jdbc-3.1.1-1.17.jar
> Reporter: macdoor615
> Assignee: david radley
> Priority: Critical
> Attachments: flink-connector-jdbc-3.0.0-1.16.png,
> flink-connector-jdbc-3.1.1-1.17.png
>
>
> create table in flink with sql-client.sh
> {code:java}
> CREATE TABLE default_catalog.default_database.a (
> ip string,
> proctime as proctime()
> )
> WITH (
> 'connector' = 'datagen'
> );{code}
> create table in mysql
> {code:java}
> create table b (
> ip varchar(20),
> type int
> ); {code}
>
> Flink 1.17.1/ 1.18.0 and *flink-connector-jdbc-3.1.1-1.17.jar*
> excute in sql-client.sh
> {code:java}
> explain SELECT * FROM default_catalog.default_database.a left join
> bnpmp_mysql_test.gem_tmp.b FOR SYSTEM_TIME AS OF a.proctime on b.type = 0 and
> a.ip = b.ip; {code}
> get the execution plan
> {code:java}
> ...
> == Optimized Execution Plan ==
> Calc(select=[ip, PROCTIME_MATERIALIZE(proctime) AS proctime, ip0, type])
> +- LookupJoin(table=[bnpmp_mysql_test.gem_tmp.b], joinType=[LeftOuterJoin],
> lookup=[ip=ip], select=[ip, proctime, ip, CAST(0 AS INTEGER) AS type, CAST(ip
> AS VARCHAR(2147483647)) AS ip0])
> +- Calc(select=[ip, PROCTIME() AS proctime])
> +- TableSourceScan(table=[[default_catalog, default_database, a]],
> fields=[ip]){code}
>
> excute same sql in sql-client with Flink 1.17.1/ 1.18.0 and
> *flink-connector-jdbc-3.0.0-1.16.jar*
> get the execution plan
> {code:java}
> ...
> == Optimized Execution Plan ==
> Calc(select=[ip, PROCTIME_MATERIALIZE(proctime) AS proctime, ip0, type])
> +- LookupJoin(table=[bnpmp_mysql_test.gem_tmp.b], joinType=[LeftOuterJoin],
> lookup=[type=0, ip=ip], where=[(type = 0)], select=[ip, proctime, ip, CAST(0
> AS INTEGER) AS type, CAST(ip AS VARCHAR(2147483647)) AS ip0])
> +- Calc(select=[ip, PROCTIME() AS proctime])
> +- TableSourceScan(table=[[default_catalog, default_database, a]],
> fields=[ip]) {code}
> with flink-connector-jdbc-3.1.1-1.17.jar, the condition is
> *lookup=[ip=ip]*
> with flink-connector-jdbc-3.0.0-1.16.jar , the condition is
> *lookup=[type=0, ip=ip], where=[(type = 0)]*
>
> In out real world production environment, this lead incorrect data output
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)