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

Jane Chan commented on FLINK-33171:
-----------------------------------

[~libenchao], Thanks for the detailed explanation. 

If I understand correctly, the current situation is that although FlinkPlanner 
turns off implicit type coercion by default, there are still issues related to 
it in practice. Some queries may fail to run, some may run but produce 
incorrect results, and some may run correctly (due to code generation support). 
In reality, there is no consistent behavior regarding implicit type coercion.

In the ideal state, after FLIP-154 is completed, if implicit type coercion is 
disabled, the validator should throw an exception directly. If type coercion is 
enabled, the type passed into code generation should already be the type after 
the implicit cast. In either case, ScalarOperatorGens may need to be 
refactored. Considering this situation, I actually think that including the 
discovered corner cases in the test suite would be helpful for future 
refactoring. What do you think? Also, I would like to ping [~snuyanzin] and ask 
about the progress of the [Revival of 
FLIP-154|https://www.mail-archive.com/[email protected]/msg68233.html].

> Table SQL support Not Equal for TimePoint type and TimeString
> -------------------------------------------------------------
>
>                 Key: FLINK-33171
>                 URL: https://issues.apache.org/jira/browse/FLINK-33171
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.18.0, 1.17.1
>            Reporter: Feng Jiajie
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.17.2, 1.18.1
>
>
> When executing the following SQL:
> {code:sql}
> SELECT
>     time1,
>     time1 = '2023-09-30 18:22:42.123' AS eq1,
>     NOT (time1 = '2023-09-30 18:22:42.123') AS notEq1
> FROM table1;
> {code}
> the result is as follows:
> {code:java}
> +----+-------------------------+--------+--------+
> | op |                   time1 |    eq1 | notEq1 |
> +----+-------------------------+--------+--------+
> | +I | 2023-09-30 18:22:42.123 |   TRUE |   TRUE |
> | +I | 2023-09-30 18:22:42.124 |  FALSE |   TRUE |
> +----+-------------------------+--------+--------+
> 2 rows in set
> {code}
> The "notEq1" in the first row should be FALSE.
> Here is the reproducing code:
> {code:java}
> import org.apache.flink.api.common.functions.RichMapFunction;
> import org.apache.flink.api.common.typeinfo.TypeInformation;
> import org.apache.flink.api.common.typeinfo.Types;
> import org.apache.flink.api.java.typeutils.RowTypeInfo;
> import org.apache.flink.configuration.Configuration;
> import org.apache.flink.streaming.api.datastream.DataStreamSource;
> import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
> import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
> import org.apache.flink.table.api.DataTypes;
> import org.apache.flink.table.api.Schema;
> import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> import org.apache.flink.types.Row;
> public class TimePointNotEqualTest {
>     public static void main(String[] args) throws Exception {
>         StreamExecutionEnvironment env =
>                 StreamExecutionEnvironment.getExecutionEnvironment(new 
> Configuration());
>         env.setParallelism(1);
>         DataStreamSource<Long> longDataStreamSource = env.fromSequence(0, 1);
>         RowTypeInfo rowTypeInfo =
>                 new RowTypeInfo(new TypeInformation[] {Types.LONG}, new 
> String[] {"time1"});
>         SingleOutputStreamOperator<Row> map =
>                 longDataStreamSource.map(new RichMapFunction<Long, Row>() {
>                     @Override
>                     public Row map(Long value) {
>                         Row row = new Row(1);
>                         row.setField(0, 1696069362123L + value);
>                         return row;
>                     }
>                 }, rowTypeInfo);
>         StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
>         Schema schema = Schema.newBuilder()
>                 .column("time1", 
> DataTypes.TIMESTAMP_LTZ(3).bridgedTo(Long.class))
>                 .build();
>         tableEnv.createTemporaryView("table1", map, schema);
>         tableEnv.sqlQuery("SELECT "
>                 + "time1," // 2023-09-30 18:22:42.123
>                 + "time1 = '2023-09-30 18:22:42.123' AS eq1," // expect TRUE
>                 + "NOT (time1 = '2023-09-30 18:22:42.123') AS notEq1 " // 
> expect FALSE but TRUE
>                 + "FROM table1").execute().print();
>     }
> }
> {code}
> I would like to attempt to fix this issue. If possible, please assign the 
> issue to me. Thank you.



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

Reply via email to