This is an automated email from the ASF dual-hosted git repository.
dwysakowicz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 2485461efed [FLINK-37803][table] Fix SQL serialization when using
LocalTime value literal without seconds (#26555)
2485461efed is described below
commit 2485461efed33dacce3ee173ec57da7141db4562
Author: Mika Naylor <[email protected]>
AuthorDate: Thu May 15 09:39:03 2025 +0200
[FLINK-37803][table] Fix SQL serialization when using LocalTime value
literal without seconds (#26555)
---
.../table/expressions/ValueLiteralExpression.java | 8 +++--
.../table/api/QueryOperationTestPrograms.java | 38 ++++++++++++++++++----
.../LiteralExpressionsSerializationITCase.java | 9 +++++
3 files changed, 46 insertions(+), 9 deletions(-)
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java
index 16a80129cd4..6367dd4b0a2 100644
---
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java
@@ -48,6 +48,7 @@ import java.time.LocalTime;
import java.time.OffsetDateTime;
import java.time.Period;
import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
@@ -271,12 +272,15 @@ public final class ValueLiteralExpression implements
ResolvedExpression {
case DATE:
return String.format("DATE '%s'",
getValueAs(LocalDate.class).get());
case TIME_WITHOUT_TIME_ZONE:
- return String.format("TIME '%s'",
getValueAs(LocalTime.class).get());
+ final LocalTime localTime = getValueAs(LocalTime.class).get();
+ return String.format(
+ "TIME '%s'",
localTime.format(DateTimeFormatter.ISO_LOCAL_TIME));
case TIMESTAMP_WITHOUT_TIME_ZONE:
final LocalDateTime localDateTime =
getValueAs(LocalDateTime.class).get();
return String.format(
"TIMESTAMP '%s %s'",
- localDateTime.toLocalDate(),
localDateTime.toLocalTime());
+ localDateTime.toLocalDate(),
+
localDateTime.toLocalTime().format(DateTimeFormatter.ISO_LOCAL_TIME));
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
final Instant instant = getValueAs(Instant.class).get();
if (instant.getNano() % 1_000_000 != 0) {
diff --git
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/QueryOperationTestPrograms.java
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/QueryOperationTestPrograms.java
index c7ee877af3e..e4ac1f12303 100644
---
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/QueryOperationTestPrograms.java
+++
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/QueryOperationTestPrograms.java
@@ -35,6 +35,7 @@ import org.apache.flink.types.RowKind;
import java.math.BigDecimal;
import java.time.Instant;
import java.time.LocalDateTime;
+import java.time.LocalTime;
import java.time.ZoneId;
import java.util.Collections;
@@ -81,15 +82,38 @@ public class QueryOperationTestPrograms {
TableTestProgram.of("values-query-operation", "verifies sql
serialization")
.setupTableSink(
SinkTestStep.newBuilder("sink")
- .addSchema("a bigint", "b string")
- .consumedValues(Row.of(1L, "abc"),
Row.of(2L, "cde"))
+ .addSchema("a bigint", "b string", "c
time", "d timestamp")
+ .consumedValues(
+ Row.of(
+ 1L,
+ "abc",
+ LocalTime.of(12, 30, 0),
+ LocalDateTime.of(1970, 1,
1, 12, 30, 0)),
+ Row.of(
+ 2L,
+ "cde",
+ LocalTime.of(18, 0, 0),
+ LocalDateTime.of(1970, 1,
1, 18, 0, 0)))
.build())
- .runTableApi(t -> t.fromValues(row(1L, "abc"), row(2L,
"cde")), "sink")
+ .runTableApi(
+ t ->
+ t.fromValues(
+ row(
+ 1L,
+ "abc",
+ LocalTime.of(12, 30, 0),
+ LocalDateTime.of(1970, 1,
1, 12, 30, 0)),
+ row(
+ 2L,
+ "cde",
+ LocalTime.of(18, 0, 0),
+ LocalDateTime.of(1970, 1,
1, 18, 0, 0))),
+ "sink")
.runSql(
- "SELECT `$$T_VAL`.`f0`, `$$T_VAL`.`f1` FROM
(VALUES \n"
- + " (CAST(1 AS BIGINT), 'abc'),\n"
- + " (CAST(2 AS BIGINT), 'cde')\n"
- + ") $$T_VAL(`f0`, `f1`)")
+ "SELECT `$$T_VAL`.`f0`, `$$T_VAL`.`f1`,
`$$T_VAL`.`f2`, `$$T_VAL`.`f3` FROM (VALUES \n"
+ + " (CAST(1 AS BIGINT), 'abc', TIME
'12:30:00', TIMESTAMP '1970-01-01 12:30:00'),\n"
+ + " (CAST(2 AS BIGINT), 'cde', TIME
'18:00:00', TIMESTAMP '1970-01-01 18:00:00')\n"
+ + ") $$T_VAL(`f0`, `f1`, `f2`, `f3`)")
.build();
static final TableTestProgram FILTER_QUERY_OPERATION =
diff --git
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/expressions/LiteralExpressionsSerializationITCase.java
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/expressions/LiteralExpressionsSerializationITCase.java
index 4a0eb653a7b..05eebac2dd6 100644
---
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/expressions/LiteralExpressionsSerializationITCase.java
+++
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/expressions/LiteralExpressionsSerializationITCase.java
@@ -57,8 +57,11 @@ public class LiteralExpressionsSerializationITCase {
void testSqlSerialization() {
final TableEnvironment env =
TableEnvironment.create(EnvironmentSettings.inStreamingMode());
final LocalTime localTime = LocalTime.of(12, 12, 12).plus(333,
ChronoUnit.MILLIS);
+ final LocalTime localTimeWithoutSeconds = LocalTime.of(12, 12);
final LocalDate localDate = LocalDate.of(2024, 2, 3);
final LocalDateTime localDateTime = LocalDateTime.of(localDate,
localTime);
+ final LocalDateTime localDateTimeWithoutSeconds =
+ LocalDateTime.of(localDate, localTimeWithoutSeconds);
final Instant instant = Instant.ofEpochMilli(1234567);
final Duration duration =
Duration.ofDays(99).plusSeconds(34).plusMillis(999);
final Period period = Period.ofMonths(470);
@@ -79,7 +82,9 @@ public class LiteralExpressionsSerializationITCase {
nullOf(DataTypes.STRING()),
lit(localDate, DataTypes.DATE().notNull()),
lit(localTime, DataTypes.TIME().notNull()),
+ lit(localTimeWithoutSeconds,
DataTypes.TIME().notNull()),
lit(localDateTime,
DataTypes.TIMESTAMP(3).notNull()),
+ lit(localDateTimeWithoutSeconds,
DataTypes.TIMESTAMP(3).notNull()),
lit(instant,
DataTypes.TIMESTAMP_LTZ(3).notNull()),
lit(
duration,
@@ -114,7 +119,9 @@ public class LiteralExpressionsSerializationITCase {
+ "CAST(NULL AS VARCHAR(2147483647)),\n"
+ "DATE '2024-02-03',\n"
+ "TIME '12:12:12.333',\n"
+ + "TIME '12:12:00',\n"
+ "TIMESTAMP '2024-02-03 12:12:12.333',\n"
+ + "TIMESTAMP '2024-02-03 12:12:00',\n"
+ "TO_TIMESTAMP_LTZ(1234567, 3),\n"
+ "INTERVAL '99 00:00:34.999' DAY TO
SECOND(3),\n"
+ "INTERVAL '39-2' YEAR TO MONTH");
@@ -138,7 +145,9 @@ public class LiteralExpressionsSerializationITCase {
null,
localDate,
localTime,
+ localTimeWithoutSeconds,
localDateTime,
+ localDateTimeWithoutSeconds,
instant,
duration,
period));