You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by "PACE, JAMES" <jp...@att.com> on 2022/10/14 13:19:37 UTC

SQL Changes between 1.14 and 1.15?

We've noticed the following difference in sql when upgrading from flink 1.14.5 to 1.15.2 around characters that are escaped in an sql statement:

This statement:
  tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\"cd\"e%'");
produces a runtime error in flink 1.15.2, but executes properly in flink 1.14.5

This can be worked around by escaping the backslash, changing the statement to:
  tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\\\"cd\\\"e%'");

This code illustrates the issue:

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 TestCase3 {
    public static void main(String[] args) throws Exception {
        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
        env.setParallelism(1);

        TestData testData = new TestData();
        testData.setField1("b\"cd\"e");
        DataStream<TestData> stream = env.fromElements(testData);
        stream.print();
        final StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(env);
        tableEnvironment.createTemporaryView("testTable", stream, Schema.newBuilder().build());

        // Works with Flink 1.14.x, flink runtime errors in 1.15.2.  Uncomment to see runtime trace
        //tableEnvironment.executeSql("select *, '1' as run from testTable WHERE lower(field1) LIKE 'b\"cd\"e%'").print();
        // Works with 1.15.2
        tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\\\"cd\\\"e%'").print();

        env.execute("TestCase");
    }

    public static class TestData {
        private String field1;

        public String getField1() { return field1; }
        public void setField1(String field1) { this.field1 = field1; }
    }
}

Thanks
Jim

Re: SQL Changes between 1.14 and 1.15?

Posted by Yuxia Luo <lu...@alumni.sjtu.edu.cn>.
Thanks for raising it. It seems a bug that introduced by this pr [1]. I have created [FLINK-29651] to trace it. 

[1] https://github.com/apache/flink/pull/19001 <https://github.com/apache/flink/pull/19001>
[2] https://issues.apache.org/jira/browse/FLINK-26520 <https://issues.apache.org/jira/browse/FLINK-26520>

Best regards,
Yuxia



> 2022年10月14日 下午9:19,PACE, JAMES <jp...@att.com> 写道:
> 
> We’ve noticed the following difference in sql when upgrading from flink 1.14.5 to 1.15.2 around characters that are escaped in an sql statement:
>  
> This statement:
>   tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\"cd\"e%'");
> produces a runtime error in flink 1.15.2, but executes properly in flink 1.14.5
>  
> This can be worked around by escaping the backslash, changing the statement to:
>   tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\\\"cd\\\"e%'");
>  
> This code illustrates the issue:
>  
> 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 TestCase3 {
>     public static void main(String[] args) throws Exception {
>         final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
>         env.setParallelism(1);
>  
>         TestData testData = new TestData();
>         testData.setField1("b\"cd\"e");
>         DataStream<TestData> stream = env.fromElements(testData);
>         stream.print();
>         final StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(env);
>         tableEnvironment.createTemporaryView("testTable", stream, Schema.newBuilder().build());
>  
>         // Works with Flink 1.14.x, flink runtime errors in 1.15.2.  Uncomment to see runtime trace
>         //tableEnvironment.executeSql("select *, '1' as run from testTable WHERE lower(field1) LIKE 'b\"cd\"e%'").print();
>         // Works with 1.15.2
>         tableEnvironment.executeSql("select * from testTable WHERE lower(field1) LIKE 'b\\\"cd\\\"e%'").print();
>  
>         env.execute("TestCase");
>     }
>  
>     public static class TestData {
>         private String field1;
>  
>         public String getField1() { return field1; }
>         public void setField1(String field1) { this.field1 = field1; }
>     }
> }
>  
> Thanks
> Jim