I don't see a problem in the result. Since you are using LEFT JOIN, the NULLs are expected where there is no matching result in the right table.
Hang Ruan <ruanhang1...@gmail.com> 于2023年12月18日周一 09:39写道: > > Hi, David. > > The FLIP-377[1] is about this part. You could take a look at it. > > Best, > Hang > > [1] > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=276105768 > > > Hang Ruan <ruanhang1...@gmail.com> 于2023年12月17日周日 20:56写道: > > > Hi, David. > > > > I think you are right that the value with NULL should not be returned if > > the filter push down is closed. > > > > Maybe you should explain this sql to make sure this filter not be pushed > > down to the lookup source. > > > > I see the configuration > > 'table.optimizer.source.predicate-pushdown-enabled' relies on the class > > FilterableTableSource, which is deprecated. > > I am not sure whether this configuration is still useful for jdbc > > connector, which is using the SupportsFilterPushDown. > > > > Maybe the jdbc connector should read this configuration and return an > > empty 'acceptedFilters' in the method 'applyFilters'. > > > > Best, > > Hang > > > > David Radley <david_rad...@uk.ibm.com> 于2023年12月16日周六 01:47写道: > > > >> Hi , > >> I am working on FLINK-33365 which related to JDBC predicate pushdown. I > >> want to ensure that the same results occur with predicate pushdown as > >> without. So I am asking this question outside the pr / issue. > >> > >> I notice the following behaviour for lookup joins without predicate > >> pushdown. I was not expecting all the <NULL>s , when there is not a > >> matching join key. ’a’ is a table in paimon and ‘db’ is a relational > >> database. > >> > >> > >> > >> Flink SQL> select * from a; > >> > >> +----+--------------------------------+-------------------------+ > >> > >> | op | ip | proctime | > >> > >> +----+--------------------------------+-------------------------+ > >> > >> | +I | 10.10.10.10 | 2023-12-15 17:36:10.028 | > >> > >> | +I | 20.20.20.20 | 2023-12-15 17:36:10.030 | > >> > >> | +I | 30.30.30.30 | 2023-12-15 17:36:10.031 | > >> > >> ^CQuery terminated, received a total of 3 rows > >> > >> > >> > >> Flink SQL> select * from db_catalog.menagerie.e; > >> > >> > >> +----+--------------------------------+-------------+-------------+-------------+-------------+ > >> > >> | op | ip | type | age | > >> height | weight | > >> > >> > >> +----+--------------------------------+-------------+-------------+-------------+-------------+ > >> > >> | +I | 10.10.10.10 | 1 | 30 | > >> 100 | 100 | > >> > >> | +I | 10.10.10.10 | 2 | 40 | > >> 90 | 110 | > >> > >> | +I | 10.10.10.10 | 2 | 50 | > >> 80 | 120 | > >> > >> | +I | 10.10.10.10 | 3 | 50 | > >> 70 | 40 | > >> > >> | +I | 20.20.20.20 | 3 | 30 | > >> 80 | 90 | > >> > >> > >> +----+--------------------------------+-------------+-------------+-------------+-------------+ > >> > >> Received a total of 5 rows > >> > >> > >> > >> Flink SQL> set table.optimizer.source.predicate-pushdown-enabled=false; > >> > >> [INFO] Execute statement succeed. > >> > >> > >> > >> Flink SQL> SELECT * FROM a left join mariadb_catalog.menagerie.e FOR > >> SYSTEM_TIME AS OF a.proctime on e.type = 2 and a.ip = e.ip; > >> > >> > >> +----+--------------------------------+-------------------------+--------------------------------+-------------+-------------+-------------+-------------+ > >> > >> | op | ip | proctime | > >> ip0 | type | age | height | > >> weight | > >> > >> > >> +----+--------------------------------+-------------------------+--------------------------------+-------------+-------------+-------------+-------------+ > >> > >> | +I | 10.10.10.10 | 2023-12-15 17:38:05.169 | > >> 10.10.10.10 | 2 | 40 | 90 | > >> 110 | > >> > >> | +I | 10.10.10.10 | 2023-12-15 17:38:05.169 | > >> 10.10.10.10 | 2 | 50 | 80 | > >> 120 | > >> > >> | +I | 20.20.20.20 | 2023-12-15 17:38:05.170 | > >> <NULL> | <NULL> | <NULL> | <NULL> | > >> <NULL> | > >> > >> | +I | 30.30.30.30 | 2023-12-15 17:38:05.172 | > >> <NULL> | <NULL> | <NULL> | <NULL> | > >> <NULL> | > >> > >> Unless otherwise stated above: > >> > >> IBM United Kingdom Limited > >> Registered in England and Wales with number 741598 > >> Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU > >> > > -- Best, Benchao Li