Feng Jiajie created FLINK-33171: ----------------------------------- Summary: 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.17.1, 1.18.0 Reporter: Feng Jiajie 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)