You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "godfrey he (Jira)" <ji...@apache.org> on 2021/10/31 14:45:00 UTC

[jira] [Assigned] (FLINK-24708) `ConvertToNotInOrInRule` has a bug which leads to wrong result

     [ https://issues.apache.org/jira/browse/FLINK-24708?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

godfrey he reassigned FLINK-24708:
----------------------------------

    Assignee: JING ZHANG

> `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: JING ZHANG
>            Priority: Major
>         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.3.4#803005)