[ 
https://issues.apache.org/jira/browse/FLINK-24708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17597605#comment-17597605
 ] 

Jianhui Dong commented on FLINK-24708:
--------------------------------------

hello, [~jingzhang], [~godfreyhe] 

I had this problem recently too with flink 1.12.2, and fixed it locally. Then I 
saw the issue, and found maybe the test case you provided can not fully 
reproduce the error, there's a rule named `SimplifyFilterConditionRule` that 
would be applied optimization before the `ConvertToNotInOrInRule`, so the test 
case

 
{code:java}
SELECT * from MyTable where e in ('CTNBSmokeSensor','H388N','H389N     
','GHL-IRD','JY-BF-20YN','HC809','DH-9908N-AEP','DH-9908N') {code}
would alse produce the right result before this bug was fixed.
{code:java}
LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4])
+- LogicalFilter(condition=[SEARCH($4, 
Sarg[_UTF-16LE'CTNBSmokeSensor':VARCHAR(15) CHARACTER SET "UTF-16LE", 
_UTF-16LE'DH-9908N':VARCHAR(15) CHARACTER SET "UTF-16LE", 
_UTF-16LE'DH-9908N-AEP':VARCHAR(15) CHARACTER SET "UTF-16LE", 
_UTF-16LE'GHL-IRD':VARCHAR(15) CHARACTER SET "UTF-16LE", 
_UTF-16LE'H388N':VARCHAR(15) CHARACTER SET "UTF-16LE", _UTF-16LE'H389N     
':VARCHAR(15) CHARACTER SET "UTF-16LE", _UTF-16LE'HC809':VARCHAR(15) CHARACTER 
SET "UTF-16LE", _UTF-16LE'JY-BF-20YN':VARCHAR(15) CHARACTER SET 
"UTF-16LE"]:VARCHAR(15) CHARACTER SET "UTF-16LE")])
   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, 
source: [TestTableSource(a, b, c, d, e)]]]) {code}
Only when we tried to check if `f(e)` in the literals scope, which `f` is a 
used-defined function, `SimplifyFilterConditionRule` would be invalid, and then 
`ConvertToNotInOrInRule` would take an effect and reproduce the bug.

I'm not very sure if there's a need to add more tests for this problem and I 
just add some of my understanding of the problem. If you think this is not a 
problem, I'm ok with this.

 

> `ConvertToNotInOrInRule` has a bug which leads to wrong result
> --------------------------------------------------------------
>
>                 Key: FLINK-24708
>                 URL: https://issues.apache.org/jira/browse/FLINK-24708
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>            Reporter: Jing Zhang
>            Assignee: Shengkai Fang
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.13.6, 1.14.3, 1.15.0
>
>         Attachments: image-2021-10-29-23-59-48-074.png
>
>
> A user report this bug in maillist, I paste the content here.
> We are in the process of upgrading from Flink 1.9.3 to 1.13.3.  We have 
> noticed that statements with either where UPPER(field) or LOWER(field) in 
> combination with an IN do not always evaluate correctly. 
>  
> The following test case highlights this problem.
>  
>  
> {code:java}
> import org.apache.flink.streaming.api.datastream.DataStream;
> import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
> import org.apache.flink.table.api.Schema;
> import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> public class TestCase {
>     public static void main(String[] args) throws Exception {
>         final StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>         env.setParallelism(1);
>         TestData testData = new TestData();
>         testData.setField1("bcd");
>         DataStream<TestData> stream = env.fromElements(testData);
>         stream.print();  // To prevent 'No operators' error
>         final StreamTableEnvironment tableEnvironment = 
> StreamTableEnvironment.create(env);
>         tableEnvironment.createTemporaryView("testTable", stream, 
> Schema.newBuilder().build());
>         // Fails because abcd is larger than abc
>         tableEnvironment.executeSql("select *, '1' as run from testTable 
> WHERE lower(field1) IN ('abcd', 'abc', 'bcd', 'cde')").print();
>         // Succeeds because lower was removed
>         tableEnvironment.executeSql("select *, '2' as run from testTable 
> WHERE field1 IN ('abcd', 'abc', 'bcd', 'cde')").print();
>         // These 4 succeed because the smallest literal is before abcd
>         tableEnvironment.executeSql("select *, '3' as run from testTable 
> WHERE lower(field1) IN ('abc', 'abcd', 'bcd', 'cde')").print();
>         tableEnvironment.executeSql("select *, '4' as run from testTable 
> WHERE lower(field1) IN ('abc', 'bcd', 'abhi', 'cde')").print();
>         tableEnvironment.executeSql("select *, '5' as run from testTable 
> WHERE lower(field1) IN ('cde', 'abcd', 'abc', 'bcd')").print();
>         tableEnvironment.executeSql("select *, '6' as run from testTable 
> WHERE lower(field1) IN ('cde', 'abc', 'abcd', 'bcd')").print();
>         // Fails because smallest is not first
>         tableEnvironment.executeSql("select *, '7' as run from testTable 
> WHERE lower(field1) IN ('cdef', 'abce', 'abcd', 'ab', 'bcd')").print();
>         // Succeeds
>         tableEnvironment.executeSql("select *, '8' as run from testTable 
> WHERE lower(field1) IN ('ab', 'cdef', 'abce', 'abcdefgh', 'bcd')").print();
>         env.execute("TestCase");
>     }
>     public static class TestData {
>         private String field1;
>         public String getField1() {
>             return field1;
>         }
>         public void setField1(String field1) {
>             this.field1 = field1;
>         }
>     }
> }
> {code}
>  
> The job produces the following output:
> Empty set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              2|
> +-----+-------------------------------++--------------------------------
> 1 row in set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              3|
> +-----+-------------------------------++--------------------------------
> 1 row in set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              4|
> +-----+-------------------------------++--------------------------------
> 1 row in set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              5|
> +-----+-------------------------------++--------------------------------
> 1 row in set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              6|
> +-----+-------------------------------++--------------------------------
> 1 row in set
> Empty set
> +-----+-------------------------------++--------------------------------
> |op|                         field1|                            run|
> +-----+-------------------------------++--------------------------------
> |+I|                            bcd|                              8|
> +-----+-------------------------------++--------------------------------
> 1 row in set
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to